Reo-LEI opened a new issue #3187:
URL: https://github.com/apache/iceberg/issues/3187


   Since we upgrade the flink dependence of iceberg from 1.12.1 to 
1.13.2(#3116), I trying to submit flink sql job to a flink cluster which flink 
version is 1.12.0, and I encounter the NoSuchMethodError as follow(the full 
stack will be attached at the end):
   ```
   java.lang.NoSuchMethodError: 
org.apache.flink.table.factories.DynamicTableFactory$Context.getCatalogTable()Lorg/apache/flink/table/catalog/ResolvedCatalogTable;
   ```
   
   I reproduce that in my local env, and I found the root cause of this error 
is flink 1.13 change the return value type of 
`DynamicTableFactory$Context.getCatalogTable()` method  from `CatalogTable` to 
`ResolvedCatalogTable`. 
   
   I think we need to found a way to be compatible with flink 1.12. @openinx 
@Flyangz @kbendick 
   
   
   **Flink 1.12**
   
https://github.com/apache/flink/blob/release-1.12/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/DynamicTableFactory.java#L53
   
   **Flink 1.13**
   
https://github.com/apache/flink/blob/release-1.13/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/DynamicTableFactory.java#L74
   
   **Trace Stack**
   
![image](https://user-images.githubusercontent.com/17312872/134923440-e2d110b5-a9a2-4613-bc88-87d95641dd9a.png)
   ```
   org.apache.flink.client.program.ProgramInvocationException: The main method 
caused an error: Unable to create a sink for writing table 
'default_catalog.default_database.table_name'.
        at 
org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:366)
        at 
org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:219)
        at 
org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114)
        at 
org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:814)
        at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:246)
        at 
org.apache.flink.client.cli.CliFrontend.parseAndRun(CliFrontend.java:1056)
        at 
org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1134)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1754)
        at 
org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
        at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1134)
   Caused by: org.apache.flink.table.api.ValidationException: Unable to create 
a sink for writing table 
        at 
org.apache.flink.table.factories.FactoryUtil.createTableSink(FactoryUtil.java:156)
        at 
org.apache.flink.table.planner.delegation.PlannerBase.getTableSink(PlannerBase.scala:369)
        at 
org.apache.flink.table.planner.delegation.PlannerBase.translateToRel(PlannerBase.scala:221)
        at 
org.apache.flink.table.planner.delegation.PlannerBase.$anonfun$translate$1(PlannerBase.scala:159)
        at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
        at scala.collection.Iterator.foreach(Iterator.scala:937)
        at scala.collection.Iterator.foreach$(Iterator.scala:937)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
        at scala.collection.IterableLike.foreach(IterableLike.scala:70)
        at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
        at scala.collection.TraversableLike.map(TraversableLike.scala:233)
        at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
        at scala.collection.AbstractTraversable.map(Traversable.scala:104)
        at 
org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:159)
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1329)
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:676)
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeOperation(TableEnvironmentImpl.java:767)
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:666)
        at 
com.huya.dc.walrus.lakehouse.flink.sql.FlinkSQLSubmitter.executeSQL(FlinkSQLSubmitter.java:156)
        at 
com.huya.dc.walrus.lakehouse.flink.sql.FlinkSQLSubmitter.main(FlinkSQLSubmitter.java:112)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:349)
        ... 11 more
   Caused by: java.lang.NoSuchMethodError: 
org.apache.flink.table.factories.DynamicTableFactory$Context.getCatalogTable()Lorg/apache/flink/table/catalog/ResolvedCatalogTable;
        at 
org.apache.iceberg.flink.FlinkDynamicTableFactory.createDynamicTableSink(FlinkDynamicTableFactory.java:104)
        at 
org.apache.flink.table.factories.FactoryUtil.createTableSink(FactoryUtil.java:153)
        ... 36 more
   ```
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to