Hi all, I am looking for feedback on removing Kafka code from apache/flink:master: https://issues.apache.org/jira/browse/FLINK-30859. If it wasn't clear, this involves moving more documentation, Kafka examples, Kafka related formats, and test related code to the new repo.
Regarding the Kafka related formats, I would like to see if anyone has any objections, as it is not as straightforward as moving modules to the external repo. This required moving the confluent-avro and debezium-json code from the flink-formats into new modules. The confluent-avro part is straightforward and it is relocated under a new module called flink-formats-kafka. However, the debezium-json code required moving only the debezium part of flink-json and the artifactId has been changed to "flink-json-debezium". The code for debezium format is unfortunately scattered across multiple modules (flink-avro-confluent-registry + flink-json) leading to this situation and it is not obvious since debezium code doesn't have an explicit Kafka dependency. Thanks to Gordon for catching this! I would suggest: 1. Move the debezium part of flink-json to external repo. 2. Create a new module under the external repo called flink-formats-kafka/flink-json-debezium where code will be located under artifactId `flink-json-debezium`. Update documentation so SQL users refer to this new artifact for the pluggable format. 3. Not remove debezium code from apache/flink:master to preserve backward compatibility for the flink-json dependency. Mark the code as deprecated to signal that new code contributions should go to the external repo. Here is a PR [1] if you would like to see the changes for yourself. Any objections? [1] https://github.com/apache/flink-connector-kafka/pull/16 Best, Mason On Mon, Mar 27, 2023 at 8:26 AM Tzu-Li (Gordon) Tai <tzuli...@apache.org> wrote: > Thanks for the updates. > > So far the above mentioned issues seem to all have PRs against > apache/flink-connector-kafka now. > > To be clear, this notice isn't about discussing _what_ PRs we will be > merging or not merging - we should try to review all of them eventually. > The only reason I've made a list of PRs in the original notice is just to > make it visible which PRs we need to reopen against > apache/flink-connector-kafka due to the code removal. > > Thanks, > Gordon > > On Sun, Mar 26, 2023 at 7:07 PM Jacky Lau <liuyong...@gmail.com> wrote: > > > Hi Gordon. https://issues.apache.org/jira/browse/FLINK-31006, which is > > also > > a critical bug in kafka. it will not exit after all partitions consumed > > when jobmanager failover in pipeline mode running unbounded source. and i > > talked with @PatrickRen <https://github.com/PatrickRen> offline, don't > > have a suitable way to fix it before. and we will solved it in this week > > > > Shammon FY <zjur...@gmail.com> 于2023年3月25日周六 13:13写道: > > > > > Thanks Jing and Gordon, I have closed the pr > > > https://github.com/apache/flink/pull/21965 and will open a new one for > > > kafka connector > > > > > > > > > Best, > > > shammon FY > > > > > > > > > On Saturday, March 25, 2023, Ran Tao <chucheng...@gmail.com> wrote: > > > > > > > Thank you Gordon and all the people who have worked on the > externalized > > > > kafka implementation. > > > > I have another pr related to Kafka[1]. I will be very appreciative if > > you > > > > can help me review it in your free time. > > > > > > > > [1] https://github.com/apache/flink-connector-kafka/pull/10 > > > > > > > > Best Regards, > > > > Ran Tao > > > > > > > > > > > > Tzu-Li (Gordon) Tai <tzuli...@apache.org> 于2023年3月24日周五 23:21写道: > > > > > > > > > Thanks Jing! I missed https://github.com/apache/flink/pull/21965 > > > indeed. > > > > > > > > > > Please let us know if anything else was overlooked. > > > > > > > > > > On Fri, Mar 24, 2023 at 8:13 AM Jing Ge <j...@ververica.com.invalid > > > > > > > wrote: > > > > > > > > > > > Thanks Gordon for driving this! There is another PR related to > > Kafka > > > > > > connector: https://github.com/apache/flink/pull/21965 > > > > > > > > > > > > Best regards, > > > > > > Jing > > > > > > > > > > > > On Fri, Mar 24, 2023 at 4:06 PM Tzu-Li (Gordon) Tai < > > > > tzuli...@apache.org > > > > > > > > > > > > wrote: > > > > > > > > > > > > > Hi all, > > > > > > > > > > > > > > Now that Flink 1.17 has been released, and given that we've > > already > > > > > > synced > > > > > > > the latest Kafka connector code up to Flink 1.17 to the > > > > > > > apache/flink-connector-kafka repo (thanks to Mason and Martijn > > for > > > > most > > > > > > of > > > > > > > the effort!), we're now in the final step of completely > removing > > > the > > > > > > Kafka > > > > > > > connector code from apache/flink:main branch, tracked by > > > FLINK-30859 > > > > > [1]. > > > > > > > > > > > > > > As such, we'd like to ask that no more Kafka connector changes > > gets > > > > > > merged > > > > > > > to apache/flink:main, effective now. Going forward, all Kafka > > > > connector > > > > > > PRs > > > > > > > should be opened directly against the > > apache/flink-connector-kafka: > > > > main > > > > > > > branch. > > > > > > > > > > > > > > Meanwhile, there's a couple of "dangling" Kafka connector PRs > > over > > > > the > > > > > > last > > > > > > > 2 months that is opened against apache/flink:main: > > > > > > > > > > > > > > 1. [FLINK-31305] Propagate producer exceptions outside of > > > mailbox > > > > > > > executor [2] > > > > > > > 2. [FLINK-31049] Add support for Kafka record headers to > > > KafkaSink > > > > > [3] > > > > > > > 3. [FLINK-31262] Move kafka sql connector fat jar test to > > > > > > > SmokeKafkaITCase [4 ] > > > > > > > 4. [hotfix] Add writeTimestamp option to > > > > > > > KafkaRecordSerializationSchemaBuilder [5] > > > > > > > > > > > > > > Apart from 1. [FLINK-31305] which is a critical bug and is > > already > > > in > > > > > > > review closed to being merged, for the rest we will be reaching > > out > > > > on > > > > > > the > > > > > > > PRs to ask the authors to close the PR and reopen them against > > > > > > > apache/flink-connector-kafka:main. > > > > > > > > > > > > > > Thanks, > > > > > > > Gordon > > > > > > > > > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-30859 > > > > > > > [2] https://github.com/apache/flink/pull/22150 > > > > > > > [3] https://github.com/apache/flink/pull/22228 > > > > > > > [4] https://github.com/apache/flink/pull/22060 > > > > > > > [5] https://github.com/apache/flink/pull/22037 > > > > > > > > > > > > > > > > > > > > > > > > > > > >