[ 
https://issues.apache.org/jira/browse/KAFKA-15144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17739710#comment-17739710
 ] 

Greg Harris commented on KAFKA-15144:
-------------------------------------

[~ecomar] I believe this is the expected new behavior of the offset translation 
logic for small numbers of records. 

In this case, it may be the `offset.lag.max` default of 100 which is preventing 
offset syncs from being emitted for each mirrored record.
Can you retest this with `offset.lag.max` of 0, or with larger numbers of 
records (100s or 1000s?)

There's also a TRACE log in MirrorSourceTask which indicates when offset syncs 
are emitted: 
[https://github.com/apache/kafka/blob/5c2492bca71200806ccf776ea31639a90290d43e/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java#L243C41-L243C41]
Offset syncs are a necessary condition for the checkpoint to move forward, so 
if the offset sync for records 1-99 aren't getting to the offset sync topic 
connector, the checkpoint task won't translate those offsets.

> 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)

Reply via email to