Hi all, I am totally fine to do step 2 with 1.17. It would give us more room to do some breaking changes if we would do it with 1.18.
@Danny Thanks for the hint. Commonly, all related interfaces/classes should be graduated too. Classes like SingleThreadMultiplexSourceReaderBase and RecordEmitter used within SourceReaderBase are a little bit different. Because SingleThreadMultiplexSourceReaderBase is extended by KafkaSourceReader which is marked as @Internal. It is not necessary to graduate them just because the KafkaSource needs to be graduated. However, given that most of those classes have been marked as @PublicEvolving more than one years, it is a good opportunity to graduate those low level APIs along with KafkaSource. And this will make the change a little bit bigger than the original expectation of removing FlinkKafkaConsumer, FlinkKafkaProducer and only graduating the related KafkaSource, KafkaSink. I am struggling if It makes sense to have a FLIP for it, because the API itself will not be modified except upgrading the annotation for graduation. Best regards, Jing On Tue, Nov 1, 2022 at 9:52 AM Piotr Nowojski <pnowoj...@apache.org> wrote: > Thanks Martijn and David, > > And what about the FlinkKafkaProducer? In order to migrate it, the user has > to make sure that the transactions stored on the state are committed, and > that there are no lingering/pending transactions (lingering transactions > are blocking further reads to be readable until they timeout). The first > one can be achieved by stop-with-savepoint. Regular savepoint or checkpoint > is not good enough, because transactions are committed on > notfiyCheckpointCompleted RPC, not on the checkpoint itself > (stop-with-savepoint is considered completed only once notifications are > received). But what about the latter? What about the pending/lingering > transactions? Are they also closed in stop-with-savepoint? > > Best, > Piotrek > > wt., 1 lis 2022 o 08:51 Martijn Visser <martijnvis...@apache.org> > napisał(a): > > > Hi all, > > > > @Piotr > > When FlinkKafkaConsumer was marked as deprecated, the migration steps > were > > included in the release notes: > > > > > https://nightlies.apache.org/flink/flink-docs-master/release-notes/flink-1.14/#flink-24055httpsissuesapacheorgjirabrowseflink-24055 > > Scrolling up a bit shows how move from FlinkKafkaProducer to KafkaSink > > > > Given the importance of the Kafka connector, I think it's smart to write > a > > dedicated blog on how to migrate. > > That could also include the differences between the two implementations, > as > > David just outlined. > > > > @Jing > > SinkV2 was released with Flink 1.15, so with 1.16 released there will be > 2 > > release cycles with that API at PublicEvolving. I don't think there were > > breaking changes when the bug fixes were applied, because those bug fixes > > were also applied in a patch version (1.15.3) which can't be done unless > > they weren't breaking the API. Unless there will be work planned for 1.17 > > on the Sink, I think it can be moved to Public too. > > > > Best regards, > > > > Martijn > > > > On Tue, Nov 1, 2022 at 7:21 AM David Anderson <dander...@apache.org> > > wrote: > > > > > > > > > > [H]ow one can migrate from the > > > > FlinkKafkaConsumer/FlinkKafkaProducer to KafkaSource/KafkaSink, while > > > > preserving exactly-once guarantees etc? > > > > > > > > > The responses from Fabian Paul in [1] and [2] address the question of > how > > > to handle the migration in terms of managing the state (where the short > > > answer is "arrange for Kafka to be the source of truth"). > > > > > > Those threads don't get into the differences in behavior between the > two > > > implementations. Here I'm thinking about (1) the loss of > > > DeserializationSchema#isEndOfStream, and the fact that you can no > longer > > > dynamically determine when the input stream has finished, and (2) the > > > change to how empty partitions are handled on startup (they used to be > > > marked idle automatically, whereas now you must use withIdleness in the > > > WatermarkStrategy). > > > > > > [1] https://www.mail-archive.com/user@flink.apache.org/msg44618.html > > > [2] https://www.mail-archive.com/user@flink.apache.org/msg45864.html > > > > > > On Mon, Oct 31, 2022 at 7:32 PM Piotr Nowojski <pnowoj...@apache.org> > > > wrote: > > > > > > > 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 > > > > > > > > > > > > > > > > > > > > > > > > > > > >