[ https://issues.apache.org/jira/browse/KAFKA-12635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17340889#comment-17340889 ]
Alexis Polyzos commented on KAFKA-12635: ---------------------------------------- Hello [~yangguo1220], thanks for the PR. We built our own image with scala 2.12 and using your PR code and ported it to Strimzi (this is how we run MM2), and following the same steps as before we still get negative lag as seen below. {code:java} ➜ bin ./kafka-consumer-groups.sh --bootstrap-server source-cluster:9092 --describe --group consumer-group-1 GROUP TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID consumer-group-1 topic_1 1 1656 1656 0 consumer-group-1-7584866df4-bgbbq-consumer-group-1-58759d7c-80e4-4ccc-8c52-bd68a6847cc3 /host_1 consumer-group-1-7584866df4-bgbbq-consumer-group-1 consumer-group-1 topic_2 1 32005209 32005209 0 consumer-group-1-7584866df4-75r7h-consumer-group-1-dc965036-5739-4399-b488-728bde48a026 /host_2 consumer-group-1-7584866df4-75r7h-consumer-group-1 consumer-group-1 topic_3 1 445 445 0 consumer-group-1-7584866df4-6bzkd-consumer-group-1-3e68ed59-2316-4168-9186-039cfcad1476 /host_3 consumer-group-1-7584866df4-6bzkd-consumer-group-1 consumer-group-1 topic_4 0 198311 198311 0 consumer-group-1-7584866df4-75r7h-consumer-group-1-aba1a270-cd9d-45e7-96a9-ac2990755e05 /host_2 consumer-group-1-7584866df4-75r7h-consumer-group-1 consumer-group-1 topic_2 0 36517046 36517055 9 consumer-group-1-7584866df4-bgbbq-consumer-group-1-6416e058-c8d6-435b-a14f-583f742ba3a4 /host_1 consumer-group-1-7584866df4-bgbbq-consumer-group-1 consumer-group-1 topic_1 0 1595 1595 0 consumer-group-1-7584866df4-c4sq6-consumer-group-1-2ec422b2-cc8f-4b0c-adae-59c016cb0666 /host_4 consumer-group-1-7584866df4-c4sq6-consumer-group-1 consumer-group-1 topic_2 2 35280609 35280613 4 consumer-group-1-7584866df4-6bzkd-consumer-group-1-19d12ff4-b486-4ea9-9eb5-23a037f11f30 /host_3 consumer-group-1-7584866df4-6bzkd-consumer-group-1 consumer-group-1 topic_2 3 33808349 33808363 14 consumer-group-1-7584866df4-c4sq6-consumer-group-1-57315540-bf27-4d99-bb38-869cd836279f /host_4 consumer-group-1-7584866df4-c4sq6-consumer-group-1 consumer-group-1 topic_1 2 1652 1652 0 consumer-group-1-7584866df4-6bzkd-consumer-group-1-9349b872-b2a4-44ba-8422-fd8f757ea017 /host_3 consumer-group-1-7584866df4-6bzkd-consumer-group-1 consumer-group-1 topic_6 1 20311 20311 0 consumer-group-1-7584866df4-c4sq6-consumer-group-1-c56dd4c4-c19d-45ea-9638-d0400f5f59d8 /host_4 consumer-group-1-7584866df4-c4sq6-consumer-group-1 consumer-group-1 topic_1 3 1720 1720 0 consumer-group-1-7584866df4-75r7h-consumer-group-1-f5b6de01-f2e1-442a-89b7-5730902392f9 /host_2 consumer-group-1-7584866df4-75r7h-consumer-group-1 consumer-group-1 topic_6 0 20252 20252 0 consumer-group-1-7584866df4-6bzkd-consumer-group-1-bca10139-17ad-4062-9008-8d3bb7a5b183 /host_3 consumer-group-1-7584866df4-6bzkd-consumer-group-1 consumer-group-1 topic_5 0 3927 3927 0 consumer-group-1-7584866df4-c4sq6-consumer-group-1-cae382b5-cb6e-48e4-8409-5c2464c6303e /host_4 consumer-group-1-7584866df4-c4sq6-consumer-group-1 consumer-group-1 topic_3 2 409 409 0 consumer-group-1-7584866df4-bgbbq-consumer-group-1-2cd4551c-5cd1-4264-b397-e3fbf695fb4f /host_1 consumer-group-1-7584866df4-bgbbq-consumer-group-1 consumer-group-1 topic_5 3 3944 3944 0 consumer-group-1-7584866df4-6bzkd-consumer-group-1-6c4872e1-51dd-46d1-b8a2-68781a6bc95c /host_3 consumer-group-1-7584866df4-6bzkd-consumer-group-1 consumer-group-1 topic_6 2 20332 20332 0 consumer-group-1-7584866df4-bgbbq-consumer-group-1-5db7872e-82d2-4da5-b888-310eeac36177 /host_1 consumer-group-1-7584866df4-bgbbq-consumer-group-1 consumer-group-1 topic_4 3 198907 198907 0 consumer-group-1-7584866df4-6bzkd-consumer-group-1-723d3e61-9aae-459d-b8fb-b22012d9e3c7 /host_3 consumer-group-1-7584866df4-6bzkd-consumer-group-1 consumer-group-1 topic_3 3 394 394 0 consumer-group-1-7584866df4-c4sq6-consumer-group-1-556bf8d7-7565-43da-8001-892d0a5c2d92 /host_4 consumer-group-1-7584866df4-c4sq6-consumer-group-1 consumer-group-1 topic_5 1 3937 3937 0 consumer-group-1-7584866df4-75r7h-consumer-group-1-1c8155fc-9f43-4dee-b788-1c6c897131f1 /host_2 consumer-group-1-7584866df4-75r7h-consumer-group-1 consumer-group-1 topic_4 1 198068 198068 0 consumer-group-1-7584866df4-bgbbq-consumer-group-1-b1e30c1b-98af-4cdf-95ea-cbceb7eecffb /host_1 consumer-group-1-7584866df4-bgbbq-consumer-group-1 consumer-group-1 topic_5 2 3891 3891 0 consumer-group-1-7584866df4-bgbbq-consumer-group-1-4f042970-58a3-4255-ba7c-6ccef11c2d7c /host_1 consumer-group-1-7584866df4-bgbbq-consumer-group-1 consumer-group-1 topic_4 2 198661 198661 0 consumer-group-1-7584866df4-c4sq6-consumer-group-1-51b2805b-9ce2-43c5-9c50-29ec4e35b227 /host_4 consumer-group-1-7584866df4-c4sq6-consumer-group-1 consumer-group-1 topic_3 0 431 431 0 consumer-group-1-7584866df4-75r7h-consumer-group-1-f757eaef-4a0e-4b36-bb6d-38b7220f7c22 /host_2 consumer-group-1-7584866df4-75r7h-consumer-group-1 consumer-group-1 topic_6 3 20222 20222 0 consumer-group-1-7584866df4-75r7h-consumer-group-1-e6962bd4-15cf-4322-bc1f-c626d32f64dd /host_2 consumer-group-1-7584866df4-75r7h-consumer-group-1 ➜ bin ./kafka-consumer-groups.sh --bootstrap-server target-cluster:9092 --describe --group consumer-group-1 Consumer group 'consumer-group-1' has no active members. GROUP TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID consumer-group-1 topic_3 3 75 35 -40 - - - consumer-group-1 topic_2 3 7135020 3202517 -3932503 - - - consumer-group-1 topic_5 2 492 208 -284 - - - consumer-group-1 topic_3 1 82 34 -48 - - - consumer-group-1 topic_2 1 4831093 1472955 -3358138 - - - consumer-group-1 topic_5 0 573 165 -408 - - - consumer-group-1 topic_1 0 1378 1376 -2 - - - consumer-group-1 topic_6 3 4614 1472 -3142 - - - consumer-group-1 topic_1 2 1422 1413 -9 - - - consumer-group-1 topic_4 2 25600 3143 -22457 - - - consumer-group-1 topic_6 1 4847 1857 -2990 - - - consumer-group-1 topic_4 0 25939 3156 -22783 - - - consumer-group-1 topic_5 3 585 170 -415 - - - consumer-group-1 topic_3 2 84 38 -46 - - - consumer-group-1 topic_5 1 520 158 -362 - - - consumer-group-1 topic_3 0 95 41 -54 - - - consumer-group-1 topic_2 2 5508653 1645293 -3863360 - - - consumer-group-1 topic_2 0 6250327 1432524 -4817803 - - - consumer-group-1 topic_6 2 5015 1963 -3052 - - - consumer-group-1 topic_1 1 1460 1453 -7 - - - consumer-group-1 topic_1 3 1467 1467 0 - - - consumer-group-1 topic_4 1 25901 3084 -22817 - - - consumer-group-1 topic_4 3 25988 3227 -22761 - - - consumer-group-1 topic_6 0 4958 1943 -3015 - - - {code} Also, the following errors are filling up our logs: {code:java} 2021-05-07 15:13:11,506 ERROR Scheduler for MirrorCheckpointTask caught exception in scheduled task: refreshing end offset of each topic partition pair at target cluster (org.apache.kafka.connect.mirror.Scheduler) [Scheduler for MirrorCheckpointTask-refreshing end offset of each topic partition pair at target cluster] org.apache.kafka.common.errors.InvalidGroupIdException: To use the group management or offset commit APIs, you must provide a valid group.id in the consumer configuration. {code} We are ready to test any other code change on your PR. Thank you again. > 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)