Hi Martin,

Thank you for your invitation. The idea of adding new improvements to
either version V3.1 or V4.0 sounds appealing to me.

> if the functionality was added in a
> backwards compatible matter (meaning a new minor version would be
> sufficient).

It seems there is no backwards compatible between new Interface
KafkaDynamicSource and privious Interface KafkaSource. As this FLIP
shows, the source state is incompatible between KafkaSource and
DynamicKafkaSource so it is recommended to reset all state or reset
partial state by setting a different uid and starting the application
from nonrestore state.[1]

However, it will not influence the current job in the previous
version. For Datastream jobs, it seems there will be no impact because
they will not call the new interface unless changes are made in the
code. For table jobs, the new FLIP-246 DynamicKafkaSource is not yet
being used.


We should pay more attention if we decide to migrate to the new
DynamicKafkaSource for table API
later on.


Yours

Hongshun

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320




On Fri, Jan 26, 2024 at 6:16 AM Martijn Visser <martijnvis...@apache.org>
wrote:

> Hi everyone,
>
> The latest version of the Flink Kafka connector that's available is
> currently v3.0.2, which is compatible with both Flink 1.17 and Flink 1.18.
>
> I would like to propose to create a release which is either v3.1, or v4.0
> (see below), with compatibility for Flink 1.17 and Flink 1.18. This newer
> version would contain many improvements [1] [2] like:
>
> * FLIP-246 Dynamic Kafka Source
> * FLIP-288 Dynamic Partition Discovery
> * Rack Awareness support
> * Kafka Record support for KafkaSink
> * Misc bug fixes and CVE issues
>
> If there are no objections, I would like to volunteer as release manager.
>
> The only thing why I'm not sure if this should be a v3.1 or a v4.0, is
> because I'm not 100% sure if FLIP-246 introduces incompatible API changes
> (requiring a new major version), or if the functionality was added in a
> backwards compatible matter (meaning a new minor version would be
> sufficient). I'm looping in Hongshun Wang and Leonard Xu to help clarify
> this.
>
> There's also a discussion happening in an open PR [3] on dropping support
> for Flink 1.18 afterwards (since this PR would add support for
> RecordEvaluator, which only exists in Flink 1.19). My proposal would be
> that after either v3.1 or v4.0 is released, we would indeed drop support
> for Flink 1.18 with that PR and the next Flink Kafka connector would be
> either v4.0 (if v3.1 is the next release) or v5.0 (if v4.0 is the next
> release).
>
> Best regards,
>
> Martijn
>
> [1]
>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522&version=12353135
> [2]
>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522&version=12352917
> [3]
>
> https://github.com/apache/flink-connector-kafka/pull/76#pullrequestreview-1844645464
>

Reply via email to