[ https://issues.apache.org/jira/browse/KAFKA-12635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17339143#comment-17339143 ]
Frank Yi commented on KAFKA-12635: ---------------------------------- [~yangguo1220] I was able to repro using the steps below on 2.8.0 brokers and MM2. # Create a topic with 1 partition on the source cluster, with `retention.ms` set to something short like 10 seconds. ## `./kafka-topics.sh --bootstrap-server $SOURCE --create --topic myTopic --config 'retention.ms=10000'` # Create a consumer that consumes from the topic ## `./kafka-console-consumer.sh --bootstrap-server $SOURCE --group myConsumer --topic myTopic` # Send 100 messages to the topic. These should get consumed by the consumer. Offset for this consumer on source cluster should be 100. ## `for i in $(seq 1 100); do echo $i; done | ./kafka-console-producer.sh --bootstrap-server $SOURCE --topic myTopic` # Wait until the retention policy deletes the records # Start MM2 with `source->target.sync.group.offsets.enabled = true` # Observe on the target cluster that log-end-offset is 0, offset is 100, and lag is -100. ## `./kafka-consumer-groups.sh --bootstrap-server $TARGET --describe --group myConsumer` > Mirrormaker 2 offset sync is incorrect if the target partition is empty > ----------------------------------------------------------------------- > > Key: KAFKA-12635 > URL: https://issues.apache.org/jira/browse/KAFKA-12635 > Project: Kafka > Issue Type: Bug > Components: mirrormaker > Affects Versions: 2.7.0 > Reporter: Frank Yi > Assignee: Ning Zhang > Priority: Major > > This bug occurs when using Mirrormaker with "sync.group.offsets.enabled = > true". > If a source partition is empty, but the source consumer group's offset for > that partition is non-zero, then Mirrormaker sets the target consumer group's > offset for that partition to the literal, not translated, offset of the > source consumer group. This state can be reached if the source consumer group > consumed some records that were now deleted (like by a retention policy), or > if Mirrormaker replication is set to start at "latest". This bug causes the > target consumer group's lag for that partition to be negative and breaks > offset sync for that partition until lag is positive. > The correct behavior when the source partition is empty would be to set the > target offset to the translated offset, not literal offset, which in this > case would always be 0. > Original email thread on this issue: > https://lists.apache.org/thread.html/r7c54ee5f57227367b911d4abffa72781772d8dd3b72d75eb65ee19f7%40%3Cusers.kafka.apache.org%3E -- This message was sent by Atlassian Jira (v8.3.4#803005)