[ https://issues.apache.org/jira/browse/FLINK-6988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16146771#comment-16146771 ]
ASF GitHub Bot commented on FLINK-6988: --------------------------------------- Github user aljoscha commented on the issue: https://github.com/apache/flink/pull/4239 I did a first high-level review of the code. I think it's good so far! Before we can merge this, however, we need a few more things around it: - A section in the Kafka doc about the new exactly-once mode, how it can be configured etc. - A big disclaimer (possibly in an "alert" box) about the interplay with the transaction timeout and what the caveats there are - A section in the Javadocs about the aforementioned caveats - A check that ensures that the transaction timeout is set to a reasonably high setting (say 1 hour) when exactly-once semantics are enabled. (With an override setting that allows the user to set a lower transaction time out if they want to.) Also, this has interplay with #4616 but we can resolve that by merging them in any order and fixing up the later changes when merging. > Add Apache Kafka 0.11 connector > ------------------------------- > > Key: FLINK-6988 > URL: https://issues.apache.org/jira/browse/FLINK-6988 > Project: Flink > Issue Type: Improvement > Components: Kafka Connector > Affects Versions: 1.3.1 > Reporter: Piotr Nowojski > Assignee: Piotr Nowojski > > Kafka 0.11 (it will be released very soon) add supports for transactions. > Thanks to that, Flink might be able to implement Kafka sink supporting > "exactly-once" semantic. API changes and whole transactions support is > described in > [KIP-98|https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging]. > The goal is to mimic implementation of existing BucketingSink. New > FlinkKafkaProducer011 would > * upon creation begin transaction, store transaction identifiers into the > state and would write all incoming data to an output Kafka topic using that > transaction > * on `snapshotState` call, it would flush the data and write in state > information that current transaction is pending to be committed > * on `notifyCheckpointComplete` we would commit this pending transaction > * in case of crash between `snapshotState` and `notifyCheckpointComplete` we > either abort this pending transaction (if not every participant successfully > saved the snapshot) or restore and commit it. -- This message was sent by Atlassian JIRA (v6.4.14#64029)