[
https://issues.apache.org/jira/browse/KAFKA-15144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17739688#comment-17739688
]
Edoardo Comar commented on KAFKA-15144:
---------------------------------------
producing 1 record at a time with th console producer, while a consumer is
polling (on source) the MM2 logs report :
{{[2023-07-03 17:44:49,479] DEBUG [MirrorCheckpointConnector|task-0]
translateDownstream(edogroup,vf-mirroring-test-edo-0,0): Translated 0 (relative
to OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0,
downstreamOffset=0}) (org.apache.kafka.connect.mirror.OffsetSyncStore:161)}}
{{[2023-07-03 17:44:49,479] TRACE [MirrorCheckpointConnector|task-0] Emitting
Checkpoint\{consumerGroupId=edogroup,
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=0,
downstreamOffset=0, metadata=} (first for this partition)
(org.apache.kafka.connect.mirror.MirrorCheckpointTask:204)}}
{{...}}
{{[2023-07-03 17:44:54,510] DEBUG [MirrorCheckpointConnector|task-0]
translateDownstream(edogroup,vf-mirroring-test-edo-0,1): Translated 1 (relative
to OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0,
downstreamOffset=0}) (org.apache.kafka.connect.mirror.OffsetSyncStore:161)}}
{{[2023-07-03 17:44:54,511] TRACE [MirrorCheckpointConnector|task-0] Emitting
Checkpoint\{consumerGroupId=edogroup,
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=1,
downstreamOffset=1, metadata=} (downstream offset advanced)
(org.apache.kafka.connect.mirror.MirrorCheckpointTask:214)}}
{{...}}
{{[2023-07-03 17:45:04,547] DEBUG [MirrorCheckpointConnector|task-0]
translateDownstream(edogroup,vf-mirroring-test-edo-0,2): Translated 1 (relative
to OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0,
downstreamOffset=0}) (org.apache.kafka.connect.mirror.OffsetSyncStore:161)}}
{{[2023-07-03 17:45:04,548] TRACE [MirrorCheckpointConnector|task-0] *Skipping*
Checkpoint{consumerGroupId=edogroup,
topicPartition=source.vf-mirroring-test-edo-0, u{*}pstreamOffset=2,
downstreamOffset=1{*}, metadata=} (repeated checkpoint)
(org.apache.kafka.connect.mirror.MirrorCheckpointTask:220)}}
> MM2 Checkpoint downstreamOffset stuck to 1
> ------------------------------------------
>
> Key: KAFKA-15144
> URL: https://issues.apache.org/jira/browse/KAFKA-15144
> Project: Kafka
> Issue Type: Bug
> Components: mirrormaker
> Reporter: Edoardo Comar
> Assignee: Edoardo Comar
> Priority: Major
> Attachments: edo-connect-mirror-maker-sourcetarget.properties
>
>
> Steps to reproduce :
> 1.Start the source cluster
> 2.Start the target cluster
> 3.Start connect-mirror-maker.sh using a config like the attached
> 4.Create a topic in source cluster
> 5.produce a few messages
> 6.consume them all with autocommit enabled
>
> 7. then dump the Checkpoint topic content e.g.
> {{% bin/kafka-console-consumer.sh --bootstrap-server localhost:9192 --topic
> source.checkpoints.internal --from-beginning --formatter
> org.apache.kafka.connect.mirror.formatters.CheckpointFormatter}}
> {{{}Checkpoint{consumerGroupId=edogroup,
> topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=3,
> {*}downstreamOffset=1{*}, metadata={}}}}
>
> the downstreamOffset remains at 1, while, in a fresh cluster pair like with
> the source topic created while MM2 is running,
> I'd expect the downstreamOffset to match the upstreamOffset.
> Note that dumping the offset sync topic, shows matching initial offsets
> {{% bin/kafka-console-consumer.sh --bootstrap-server localhost:9192 --topic
> mm2-offset-syncs.source.internal --from-beginning --formatter
> org.apache.kafka.connect.mirror.formatters.OffsetSyncFormatter}}
> {{{}OffsetSync{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0,
> downstreamOffset=0{}}}}
>
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)