[ https://issues.apache.org/jira/browse/KAFKA-10339?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17171519#comment-17171519 ]
Ryanne Dolan commented on KAFKA-10339: -------------------------------------- I think what you mean is that a read-process-write loop cannot span clusters, since a transaction coordinator on one cluster cannot commit offsets on another cluster. But I don't think we actually need that -- we can just store offsets on the target cluster instead. I think what we need is something along these lines: - we manage offsets ourselves -- we don't rely on Connect's internal offsets tracking or __consumer_offsets on the source cluster. - we only write to the target cluster. - offsets are stored on the target cluster using a "fake" consumer group. I say "fake" because there would be no actual records being consumed by the group, just offsets being stored in __consumer_offsets topic. - we write all records in a transaction, just as the KIP currently describes. - in addition, we call addOffsetsToTransaction in order to commit offsets to the "fake" consumer group on the target cluster. - when MirrorSourceTask starts, it loads initial offsets from __consumer_offsets on the target cluster. Result: - if the transaction succeeds, the __consumer_offsets topic on the target cluster is updated. - if the transaction aborts, all data records are dropped, and the __consumer_offsets topic is not updated. - when MirrorSourceTask starts/restarts, it resumes at the last committed offsets, as recorded in the target cluster. Thoughts? > MirrorMaker2 Exactly-once Semantics > ----------------------------------- > > Key: KAFKA-10339 > URL: https://issues.apache.org/jira/browse/KAFKA-10339 > Project: Kafka > Issue Type: New Feature > Components: KafkaConnect > Reporter: Ning Zhang > Assignee: Ning Zhang > Priority: Major > Labels: needs-kip > > MirrorMaker2 is currently implemented on Kafka Connect Framework, more > specifically the Source Connector / Task, which do not provide exactly-once > semantics (EOS) out-of-the-box, as discussed in > https://github.com/confluentinc/kafka-connect-jdbc/issues/461, > https://github.com/apache/kafka/pull/5553, > https://issues.apache.org/jira/browse/KAFKA-6080 and > https://issues.apache.org/jira/browse/KAFKA-3821. Therefore MirrorMaker2 > currently does not provide EOS. -- This message was sent by Atlassian Jira (v8.3.4#803005)