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

Federico Valeri edited comment on KAFKA-12635 at 2/4/22, 8:51 AM:
------------------------------------------------------------------

I was able to reproduce the issue on Kafka 2.7.2 and 2.8.1, but not on 3.1.0.

State of the source cluster after producing/consuming 1mln records:

{code}
$ $KAFKA_HOME/bin/kafka-topics.sh --bootstrap-server :9090 --describe --topic 
my-topic
Topic: my-topic PartitionCount: 3       ReplicationFactor: 3    Configs: 
min.insync.replicas=2,message.format.version=2.8-IV1
        Topic: my-topic Partition: 0    Leader: 2       Replicas: 2,0,1 Isr: 
2,0,1
        Topic: my-topic Partition: 1    Leader: 0       Replicas: 0,1,2 Isr: 
0,1,2
        Topic: my-topic Partition: 2    Leader: 1       Replicas: 1,2,0 Isr: 
1,2,0

$ $KAFKA_HOME/bin/kafka-consumer-groups.sh --bootstrap-server :9090 --describe 
--group my-group

Consumer group 'my-group' has no active members.

GROUP           TOPIC           PARTITION  CURRENT-OFFSET  LOG-END-OFFSET  LAG  
           CONSUMER-ID     HOST            CLIENT-ID
my-group        my-topic        0          332600          332600          0    
           -               -               -
my-group        my-topic        1          335510          335510          0    
           -               -               -
my-group        my-topic        2          331890          331890          0    
           -               -               -
{code}

State of the target cluster after MM2 has done its job 
(sync.group.offsets.enabled = true):

{code}
$ $KAFKA_HOME/bin/kafka-topics.sh --bootstrap-server :9093 --describe --topic 
my-topic
Topic: my-topic PartitionCount: 3       ReplicationFactor: 3    Configs: 
min.insync.replicas=2,message.format.version=2.8-IV1
        Topic: my-topic Partition: 0    Leader: 3       Replicas: 3,4,5 Isr: 
3,4,5
        Topic: my-topic Partition: 1    Leader: 4       Replicas: 4,5,3 Isr: 
4,5,3
        Topic: my-topic Partition: 2    Leader: 5       Replicas: 5,3,4 Isr: 
5,3,4

$ $KAFKA_HOME/bin/kafka-consumer-groups.sh --bootstrap-server :9093 --describe 
--group my-group

Consumer group 'my-group' has no active members.

GROUP           TOPIC           PARTITION  CURRENT-OFFSET  LOG-END-OFFSET  LAG  
           CONSUMER-ID     HOST            CLIENT-ID
my-group        my-topic        0          332600          0               
-332600         -               -               -
my-group        my-topic        1          335510          0               
-335510         -               -               -
my-group        my-topic        2          331890          0               
-331890         -               -               -
{code}

There is actually no need to set a custom value for retention.ms in order to 
trigger the issue.


was (Author: fvaleri):
I was able to reproduce the issue on Kafka 2.7.2 and 2.8.1, but not on 3.1.0.

State of the source cluster after producing/consuming 1mln records:

{code}
$ $KAFKA_HOME/bin/kafka-topics.sh --bootstrap-server :9090 --describe --topic 
my-topic
Topic: my-topic PartitionCount: 3       ReplicationFactor: 3    Configs: 
min.insync.replicas=2,message.format.version=2.8-IV1
        Topic: my-topic Partition: 0    Leader: 2       Replicas: 2,0,1 Isr: 
2,0,1
        Topic: my-topic Partition: 1    Leader: 0       Replicas: 0,1,2 Isr: 
0,1,2
        Topic: my-topic Partition: 2    Leader: 1       Replicas: 1,2,0 Isr: 
1,2,0

$ $KAFKA_HOME/bin/kafka-consumer-groups.sh --bootstrap-server :9090 --describe 
--group my-group

Consumer group 'my-group' has no active members.

GROUP           TOPIC           PARTITION  CURRENT-OFFSET  LOG-END-OFFSET  LAG  
           CONSUMER-ID     HOST            CLIENT-ID
my-group        my-topic        0          332600          332600          0    
           -               -               -
my-group        my-topic        1          335510          335510          0    
           -               -               -
my-group        my-topic        2          331890          331890          0    
           -               -               -
{code}

State of the target cluster after MM2 has done its job 
(sync.group.offsets.enabled = true, replication.policy.class = 
io.strimzi.kafka.connect.mirror.IdentityReplicationPolicy):

{code}
$ $KAFKA_HOME/bin/kafka-topics.sh --bootstrap-server :9093 --describe --topic 
my-topic
Topic: my-topic PartitionCount: 3       ReplicationFactor: 3    Configs: 
min.insync.replicas=2,message.format.version=2.8-IV1
        Topic: my-topic Partition: 0    Leader: 3       Replicas: 3,4,5 Isr: 
3,4,5
        Topic: my-topic Partition: 1    Leader: 4       Replicas: 4,5,3 Isr: 
4,5,3
        Topic: my-topic Partition: 2    Leader: 5       Replicas: 5,3,4 Isr: 
5,3,4

$ $KAFKA_HOME/bin/kafka-consumer-groups.sh --bootstrap-server :9093 --describe 
--group my-group

Consumer group 'my-group' has no active members.

GROUP           TOPIC           PARTITION  CURRENT-OFFSET  LOG-END-OFFSET  LAG  
           CONSUMER-ID     HOST            CLIENT-ID
my-group        my-topic        0          332600          0               
-332600         -               -               -
my-group        my-topic        1          335510          0               
-335510         -               -               -
my-group        my-topic        2          331890          0               
-331890         -               -               -
{code}

There is actually no need to set a custom value for retention.ms in order to 
trigger the issue.

> 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.20.1#820001)

Reply via email to