Hi all,
this thought came up on FLINK-17260 [1] but I think it would be a good idea in general. The issue reminded us that Kafka didn't have an idempotent/fault-tolerant Producer before Kafka 0.11.0. By now we have had the "modern" Kafka connector that roughly follows new Kafka releases for a while and this one supports Kafka cluster versions as far back as 0.10.2.0 (I believe). What are your thoughts on removing support for older Kafka versions? And yes, I know that we had multiple discussions like this in the past but I'm trying to gauge the current sentiment. I'm cross-posting to the user-ml since this is important for both users and developers. Best, Aljoscha [1] https://issues.apache.org/jira/browse/FLINK-17260 |
Hi Aljoscha,
I'm lightly leaning towards keeping the 0.10 connector, for Kafka 0.10 still has a steady user base in my observation. But if we drop 0.10 connector, can we ensure the users would be able to smoothly migrate to 0.11 connector/universal connector? If I remember correctly, the universal connector is compatible with 0.10 brokers, but I want to double check that. Best, Paul Lam
|
Hi Aljoscha, I am assuming you're asking about dropping the flink-connector-kafka-0.10/0.11 modules, right? Or are you talking about removing support for Kafka 0.10/0.11 from the universal connector? I am in favor of removing flink-connector-kafka-0.10/0.11 in the next release. These modules would still be available in Flink 1.11- as a reference, and could be used with Flink 1.12+ with small or no modifications. To my knowledge, you also use the universal Kafka connector with 0.10 brokers, but there might be a performance penalty if I remember correctly. In general, I find it important to continuously reduce baggage that accumulates over time and this seems like a good opportunity. Best, Konstantin On Tue, Aug 25, 2020 at 4:59 AM Paul Lam <[hidden email]> wrote:
|
+1 to remove both the 1.10 and 1.11
connectors.
The connectors have not been actively
developed for some time. They are basically just sitting around
causing noise by causing test instabilities and eating CI time.
It would also allow us to really
simplify the module structure of the Kafka connectors.
Users may continue to use the 1.11
version of the connectors with future Flink versions, and we may
even provide critical bug fixes in a 1.11 bugfix release (albeit
unlikely).
While ultimately this is a separate
topic I would also be in favor of removing any migration paths we
have from 0.11 to the universal connector;
as these are already present in 1.11
users may migrate to the universal connector before jumping to
Flink 1.12+.
On 25/08/2020 18:49, Konstantin Knauf
wrote:
|
@Konstantin: Yes, I'm talking about dropping those modules. We don't
have any special code for supporting Kafka 0.10/0.11 in the "modern" connector, that comes from the Kafka Consumer/Producer code we're using. @Paul: The modern Kafka connector works with Kafka brokers as far back as 0.10, would that be enough or do you still think we should have the actual Kafka 0.10 Consumer code in Flink as well. Best, Aljoscha On 25.08.20 23:15, Chesnay Schepler wrote: > +1 to remove both the 1.10 and 1.11 connectors. > > The connectors have not been actively developed for some time. They are > basically just sitting around causing noise by causing test > instabilities and eating CI time. > It would also allow us to really simplify the module structure of the > Kafka connectors. > > Users may continue to use the 1.11 version of the connectors with future > Flink versions, and we may even provide critical bug fixes in a 1.11 > bugfix release (albeit unlikely). > > While ultimately this is a separate topic I would also be in favor of > removing any migration paths we have from 0.11 to the universal connector; > as these are already present in 1.11 users may migrate to the universal > connector before jumping to Flink 1.12+. > > On 25/08/2020 18:49, Konstantin Knauf wrote: >> Hi Aljoscha, >> >> I am assuming you're asking about dropping the >> flink-connector-kafka-0.10/0.11 modules, right? Or are you talking >> about removing support for Kafka 0.10/0.11 from the universal connector? >> >> I am in favor of removing flink-connector-kafka-0.10/0.11 in the next >> release. These modules would still be available in Flink 1.11- as a >> reference, and could be used with Flink 1.12+ with small or no >> modifications. To my knowledge, you also use the universal Kafka >> connector with 0.10 brokers, but there might be a performance >> penalty if I remember correctly. In general, I find it important >> to continuously reduce baggage that accumulates over time and this >> seems like a good opportunity. >> >> Best, >> >> Konstantin >> >> On Tue, Aug 25, 2020 at 4:59 AM Paul Lam <[hidden email] >> <mailto:[hidden email]>> wrote: >> >> Hi Aljoscha, >> >> I'm lightly leaning towards keeping the 0.10 connector, for Kafka >> 0.10 still has a steady user base in my observation. >> >> But if we drop 0.10 connector, can we ensure the users would be >> able to smoothly migrate to 0.11 connector/universal connector? >> >> If I remember correctly, the universal connector is compatible >> with 0.10 brokers, but I want to double check that. >> >> Best, >> Paul Lam >> >>> 2020年8月24日 22:46,Aljoscha Krettek <[hidden email] >>> <mailto:[hidden email]>> 写道: >>> >>> Hi all, >>> >>> this thought came up on FLINK-17260 [1] but I think it would be a >>> good idea in general. The issue reminded us that Kafka didn't >>> have an idempotent/fault-tolerant Producer before Kafka 0.11.0. >>> By now we have had the "modern" Kafka connector that roughly >>> follows new Kafka releases for a while and this one supports >>> Kafka cluster versions as far back as 0.10.2.0 (I believe). >>> >>> What are your thoughts on removing support for older Kafka >>> versions? And yes, I know that we had multiple discussions like >>> this in the past but I'm trying to gauge the current sentiment. >>> >>> I'm cross-posting to the user-ml since this is important for both >>> users and developers. >>> >>> Best, >>> Aljoscha >>> >>> [1] https://issues.apache.org/jira/browse/FLINK-17260 >> >> >> >> -- >> >> Konstantin Knauf >> >> https://twitter.com/snntrable >> >> https://github.com/knaufk >> > > |
Hi,
I think it’s okay, given that we can either migrate to the universal connector or still use the compatible 0.10/0.11 connector of 1.11 release as Chesnay mentioned when upgrading to 1.12. IIUC, the migration process to the universal connector would be (please correct me if I’m wrong): 1. Stop the job with a savepoint, committing the offset to Kafka brokers. 2. Modify user code, migrate to he universal connector, and change the source operator id to discard the old connector states. 3. Start the job with the savepoint, and read Kafka from group offsets. Best, Paul Lam
|
Free forum by Nabble | Edit this page |