You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/09/28 04:05:49 UTC

[GitHub] [iceberg] openinx commented on issue #3187: Flink: Get NoSuchMethodError when submit flink sql by flink 1.12.x since iceberg upgrade flink dependence to 1.13.2

openinx commented on issue #3187:
URL: https://github.com/apache/iceberg/issues/3187#issuecomment-928755046


   Thanks for the report, @Reo-LEI !   I think this issue was introduced from this [apache flink PR](https://github.com/apache/flink/pull/15316/files#diff-bd276ed951054125b39428ee61de103d9c7832246398f01514a574bb8e51757cR74) and [FLINK-21913](https://issues.apache.org/jira/browse/FLINK-21913),  it just changed the returned data type from `CatalogTable` to `ResolvedCatalogTable` without any compatibility guarantee.  In this case,  the iceberg-flink-runtime jar which is compiled from apache flink 1.13 will include the `ResovledCatalogTable` class inside it.  Finally when we   package this jar and submit the flink job to flink 1.12,  the above compatibility issue happen.
   
   As we all know, the [DynamicTableFactory](https://github.com/apache/flink/blob/99c2a415e9eeefafacf70762b6f54070f7911ceb/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/DynamicTableFactory.java) is a basic API which almost all flink connectors are built on top of it.  The breaking compatibility makes the downstream projects really hard to deliver better compatibility to users, unless we iceberg maintain different modules for each maintained flink version (That's not the thing that we want to do).   
   
   The last flink upgrading work is also not a good experience (See the [discussion](https://github.com/apache/iceberg/pull/1956) and [comment](https://github.com/apache/iceberg/pull/1956#discussion_r546534299) ),  because the flink 1.12 also breaks several API that was annotated `PublicEvolving` in flink 1.11.0,  that becomes one of the most important reasons leading to the conclusion that stops support flink 1.11.0 in our apache iceberg branch ( Supporting new features [such as flip-27 unified iceberg source/sink] that depends the API introduced in flink 1.12  is another reason).   To better support the compatibility of downstream systems and delivering better experience to flink users,  I will strongly suggest the Apache Flink community to pay more attention to ensuring API compatibility.   
   
   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org