Hi,

Maybe a stupid question, but how one can migrate from the
FlinkKafkaConsumer/FlinkKafkaProducer to KafkaSource/KafkaSink, while
preserving exactly-once guarantees etc? Is it possible? I've tried a quick
search and couldn't find it, but maybe I was looking in wrong places.

Best,
Piotrek

pon., 31 paź 2022 o 16:40 Jing Ge <j...@ververica.com> napisał(a):

> Thanks Martijn. What you said makes a lot of sense. I figure we should do
> it in 2 steps.
>
>  Step 1 (with 1.17):
> - Remove FlinkKafkaConsumer.
> - Graduate Kafka Source from @PublicEvolving to @Public.
> - Update doc and leave hints for customers as the reference.
>
> According to [1], the Kafka Sink should also be graduated with 1.17, i.e.
> after 1.15 and 1.16 two release cycles. But since the design change from
> SinkV1 to SinkV2 were significant and there were many change requests since
> then, we'd better give the sink one more release cycle time to become more
> stable. The other reason for giving the Sink more time is that the
> experimental phase was only covered by one release cycle instead of two as
> [1] suggested.
>
> Step 2 (with 1.18 ):
> - Remove FlinkKafkaProducer.
> - Graduate Kafka Sink from @PublicEvolving to @Public.
> - Update doc and leave hints for customers as the reference.
>
> Best regards,
> Jing
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-197%3A+API+stability+graduation+process
>
> On Thu, Oct 27, 2022 at 3:01 PM Martijn Visser <martijnvis...@apache.org>
> wrote:
>
> > Hi Jing,
> >
> > Thanks for opening the discussion. I see no issue with removing the
> > FlinkKafkaConsumer, since it has been marked as deprecated and the Source
> > API (which is used by the KafkaSource) is marked as @Public (at least the
> > Base implementation)
> >
> > The successor of the FlinkKafkaProducer is the KafkaSink, which is using
> > the Sink V2 API which is still marked as @PublicEvolving (Base
> > implementation). I think that we should only remove the
> FlinkKafkaProducer
> > if we also mark the Sink V2 as @Public. I don't think that should be a
> > problem (since it's based on the first Sink implementation, which was
> > Experimental in 1.14 and got replaced with Sink V2 as PublicEvolving in
> > 1.15).
> >
> > Thanks,
> >
> > Martijn
> >
> > On Thu, Oct 27, 2022 at 2:06 PM Jing Ge <j...@ververica.com> wrote:
> >
> > > Hi Dev,
> > >
> > > I'd like to start a discussion about removing FlinkKafkaConsumer and
> > > FlinkKafkaProducer in 1.17.
> > >
> > > Back in the past, it was originally announced to remove it with Flink
> > 1.15
> > > after Flink 1.14 had been released[1]. And then postponed to the next
> > 1.15
> > > release which meant to remove it with Flink 1.16 but forgot to change
> the
> > > doc[2]. I have created a PRs to fix it. Since the 1.16 release branch
> has
> > > code freeze, it makes sense to, first of all, update the doc to say
> that
> > > FlinkKafkaConsumer will be removed with Flink 1.17 [3][4] and second
> > start
> > > the discussion about removing them with the current master branch i.e.
> > for
> > > the coming 1.17 release. I'm all ears and looking forward to your
> > feedback.
> > > Thanks!
> > >
> > > Best regards,
> > > Jing
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > [1]
> > >
> > >
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/connectors/datastream/kafka/#kafka-sourcefunction
> > > [2]
> > >
> > >
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/datastream/kafka/#kafka-sourcefunction
> > > [3] https://github.com/apache/flink/pull/21172
> > > [4] https://github.com/apache/flink/pull/21171
> > >
> >
>

Reply via email to