Thanks for the work, Arvid! I'm not sure if there's any incompatibility issue for 3.2 + 1.20. If they are fully compatible, what about not dropping support for 1.18 in 3.2.1, and we release one more version 3.2.1-1.20? Then we can use 3.3.0 for the new lineage feature in 1.20 and drop support for 1.18 and 1.19.
And for the 4.0.0-preview version I'd like to help with it :-) Best, Qingsheng On Fri, Sep 27, 2024 at 6:13 PM Arvid Heise <ar...@apache.org> wrote: > > Hi David, > > thank you very much for your reply. > > > Some thoughts on whether we need the 3 deliverables. And whether we could > follow more traditional fixpack numbering: > > I see that there is already a release for 1.19 > https://mvnrepository.com/artifact/org.apache.flink/flink-connector-kafka/3.2.0-1.19 > . > I am wondering why we need the first deliverable. If we need it for the bug > fixes , why not have a 3.2.1? > I forgot the most important part of the first release: drop Flink 1.18 > support and add Flink 1.20 support. Hence I wouldn't want to mix that into > a bugfix release. I think this is in line with the previous minor releases. > > > I assume that kafka-3.4.0 will not work with previous Flink releases. > Would it be worth have a config switch to enable the lineage in the > connector so that we could use it with 1.19? We could maybe do a 3.3 if > this was the case. > Yes, as outlined in the discussion linked in the original message, we need > to mixin new interfaces. Afaik classloading will fail if the interfaces are > not present, even if the methods are not used. So I don't see how we can > use a config switch to make it happen (except with code duplication). > However, I'm grateful for any ideas to avoid this release. > > Best, > > Arvid > > On Fri, Sep 27, 2024 at 11:11 AM David Radley <david_rad...@uk.ibm.com> > wrote: > > > Hi Arvid, > > Some thoughts on whether we need the 3 deliverables. And whether we could > > follow more traditional fixpack numbering: > > I see that there is already a release for 1.19 > > https://mvnrepository.com/artifact/org.apache.flink/flink-connector-kafka/3.2.0-1.19 > > . I am wondering why we need the first deliverable. If we need it for the > > bug fixes , why not have a 3.2.1? > > I assume that kafka-3.4.0 will not work with previous Flink releases. > > Would it be worth have a config switch to enable the lineage in the > > connector so that we could use it with 1.19? We could maybe do a 3.3 if > > this was the case. > > > > WDYT? > > Kind regards, David. > > > > > > > > From: Arvid Heise <ar...@apache.org> > > Date: Friday, 27 September 2024 at 09:24 > > To: dev@flink.apache.org <dev@flink.apache.org> > > Subject: [EXTERNAL] Kafka connector releases > > Dear Flink devs, > > > > I'd like to initiate three(!) Kafka connector releases. The main reason for > > having three releases is that we have been slacking a bit in keeping up > > with the latest changes. > > > > Here is the summary: > > 1. Release kafka-3.3.0 targeting 1.19 and 1.20 (asap) > > - Incorporates lots of deprecations for Flink 2 including everything that > > is related to FlinkKafkaProducer (SinkFunction, FlinkKafkaConsumer > > (SourceFunction), and KafkaShuffle > > - Lots of bugfixes that are very relevant for 1.19 users (and probably work > > with older releases) > > > > 2. Release kafka-3.4.0 targeting 1.20 (~1-2 weeks later) > > - Incorporates lineage tracing which is only available in 1.20 [1] (FLIP > > incorrectly says that it's avail in 1.19) > > - We have discussed some alternatives to this release in [2] but basically > > having a separate release is the cleanest solution. > > - I'd like to linearize the releases to avoid having to do back or even > > forward ports > > > > 3. Release kafka-4.0.0 targeting 2.0-preview (~1-2 weeks later) > > - Much requested to get the connector out asap for the preview. (I think > > the old jar using the removed interfaces should still work) > > - Remove all deprecated things > > - General spring cleaning (trying to get rid of arch unit violations, > > migrate to JUnit5) > > - Should we relocate the TableAPI stuff to o.a.f.connectors? > > > > I'd appreciate any feedback and volunteers for RM ;) If you have pending > > PRs that should be part of any of those releases, please also write them. > > > > Best, > > > > Arvid > > > > [1] > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-314%3A+Support+Customized+Job+Lineage+Listener > > [1] > > > > https://github.com/apache/flink-connector-kafka/pull/111#issuecomment-2306382878 > > > > Unless otherwise stated above: > > > > IBM United Kingdom Limited > > Registered in England and Wales with number 741598 > > Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 3AU > >