pvary commented on issue #8930: URL: https://github.com/apache/iceberg/issues/8930#issuecomment-1783722052
> > then we will have a transitive dependency on Flink 1.18. We can exclude the dependency, but how can we make sure that everything works as expected without running the tests. > > You're right. There'll be a transitive dependency. If Flink1.17's APIs are incompatible with Flink1.18, I think we can copy some tests into the Flink1.17 module to ensure it works fine and add end-to-end tests to guarantee compatibility with different Flink versions I think the issue here, is that we can not be sure, which dependency needs to be copied over in advance. I tried to upgrade to Flink 1.18 yesterday, and found that there is a Flink change which causes the TestFlinkMetaDataTable#testAllFilesPartitioned test to fail. See: https://apache-flink.slack.com/archives/C03G7LJTS2G/p1698402761715879?thread_ts=1698402761.715879&cid=C03G7LJTS2G One can argue that these issues are found at upgrade time, but if we add features between migration, then we will not find them, only if we run all of the tests. So I think running all of the tests on every supported Flink version is a must. > > As an alternative we can move these classes around every time when it is needed - but this approach has its own maintenance costs. > > As of Flink 1.18, released a few days ago, the Flink community has externalized the [JDBC](https://github.com/apache/flink-connector-jdbc), [ES](https://github.com/apache/flink-connector-elasticsearch), [Kafka](https://github.com/apache/flink-connector-kafka), [pulsar](https://github.com/apache/flink-connector-pulsar), [HBase](https://github.com/apache/flink-connector-hbase), and so on connectors. That means the Flink API has become more compatible. So I think this maintenance cost between different Flink versions is acceptable. I am currently working on extending the Flink Sink API to provide features needed for Iceberg. See: - https://cwiki.apache.org/confluence/display/FLINK/FLIP-371%3A+Provide+initialization+context+for+Committer+creation+in+TwoPhaseCommittingSink - https://cwiki.apache.org/confluence/display/FLINK/FLIP-372%3A+Allow+TwoPhaseCommittingSink+WithPreCommitTopology+to+alter+the+type+of+the+Committable I would like to use them ASAP, and this will cause differences between 1.18/1.19 versions of the Iceberg connector. If we have flink-common, then until 1.18 this should be in common, after 1.19 it should go to the version specific code, and after we drop support for 1.18 then it will go to the common code again. Also we would need to come up with the appropriate abstractions for separate the changing code from the fix code. These issues might be even more pronounced when the Flink 2.0 comes out, of which the discussion is already started. IMHO these are more complex tasks than simply backporting the changes to the other branches. Maybe that is the cause why the Iceberg-Spark code is handled in the same way. -- 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-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org