Hi Danny,

thanks for bringing this to my attention. I'm also in favor of starting
with Flink 2.0. It's just a nice opportunity to break with some traditions
and we are almost there anyways. We could resume the discussion over there
if you'd like.

Regarding the deprecation. I think you misunderstood me. I was talking
about deprecating the KafkaShuffle and all the old API around
Consumer/Producer (which we already deprecated in 1.14). [1]
But I'd also deprecate 1.18 support with that release, so all that you say
is additionally true.

Since we are on the topic of deprecation and I got your attention: Does
anyone have strong opinions about moving the TableAPI stuff into the
o.a.f.connectors package? It would probably moving things + provide stubs
with deprecation in the old location that extend from the actual things
similar to the FlinkKafkaPartitioner in [1]. We can start a separate
discussion and I don't have a strong opinion since I'm not a TableAPI/SQL
guy. I'm also getting low on OSS contribution time.

Best,

Arvid

[1] https://github.com/apache/flink-connector-kafka/pull/121

On Mon, Sep 30, 2024 at 4:49 PM Danny Cranmer <dannycran...@apache.org>
wrote:

> Thanks for driving this Arvid.
>
> +1 for the releases and the RM
>
> For the 4.0.0 version it would be good to drop the Flink version from the
> connector version (4.0.0, not 4.0.0-2.0). I started a discussion some time
> ago [1], however the thread went stale. But I think the feedback was
> generally aligned. I would propose we release 4.0.0 (or 4.0.0-preview?) and
> document this only supports Flink 2.0-preview.
>
> > I don't think we have any precedent of deprecating API in a bugfix
> release.
>
> I have not seen us do that before for connectors. I would personally prefer
> a 3.3.0 if we are dropping Flink 1.18 support, although I agree it is a
> grey area in the guidelines. This allows us to more easily create a
> 3.2.1-1.18 hotfix later incase the need arises and the community agrees to
> push another 1.18 hotfix.
>
> >  It's not obvious to me if a bugfix release counts towards our
> deprecation policy where we have one release between deprecation and
> removal.
>
> Given that we encode the Flink version in the connector version, it is
> difficult/impossible to not notice it has been deprecated. Since the user
> would bump the version from 3.2.0-1.18 to 3.3.0-1.18, and this artifact
> would not be found in Maven central, and fail the build.
>
> > 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.
>
> This is a possibility, but still results in 3 releases, so only worth it if
> it is simpler
>
> Thanks,
> Danny
>
> [1] https://lists.apache.org/thread/byy8fgkr8dbrrdv2nxpxkrz18h904r9b
>
>
> On Mon, Sep 30, 2024 at 10:36 AM Arvid Heise <ar...@apache.org> wrote:
>
> > Hi folks,
> >
> > thanks for the feedback. I'm still not convinced that it's the best
> > approach to start with a bugfix release but I did a poor job in
> > highlighting my thoughts.
> > * I don't think we have any precedent of deprecating API in a bugfix
> > release.
> > * It's not obvious to me if a bugfix release counts towards our
> deprecation
> > policy where we have one release between deprecation and removal. A user
> > could use 3.2.0 not notice the 3.2.1 and see the deprecation only in
> 3.3.0.
> > Then the user could be surprised by the removal in 4.0.0. (I'd personally
> > see our policy too strict regarding major releases but that's what we did
> > for Flink 2)
> > * For the deprecation, I also introduced a new interface KafkaPartitioner
> > that's a drop-in replacement for FlinkKafkaPartitioner. This is new API
> and
> > should be part of a new minor release.
> > * Finally, there is one new feature: [FLINK-22748] Allow dynamic target
> > topic selection in SQL Kafka sinks [1].
> > * I'd drop 1.18 still and support 1.20 instead.
> > * I'd also bump kafka-clients to 3.7.1
> >
> > WDYT?
> >
> > Best,
> >
> > Arvid
> >
> > [1] https://github.com/apache/flink-connector-kafka/pull/109
> >
> > On Sat, Sep 28, 2024 at 5:27 AM Leonard Xu <xbjt...@gmail.com> wrote:
> >
> > > Thanks Arvid for the volunteering!
> > >
> > > + 1 for all of the three releases and RM candidate.Qingsheng and I
> would
> > > like to help the 4.0.0-preview which follows the Flink 2.0 preview,
> > > please feel free to ping us if you need any help.
> > >
> > > Btw, for other external connectors which  highest supported flink
> version
> > > still is 1.17 or 1.18,  I also hope we can have dedicated plan to bump
> > > their version ASAP, we can
> > > start a new thread to track other external connector releases.
> > >
> > > Best,
> > > Leonard
> > >
> > >
> > > > 2024年9月27日 下午6:54,Qingsheng Ren <re...@apache.org> 写道:
> > > >
> > > > 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
> > > >>>
> > >
> > >
> >
>

Reply via email to