[ https://issues.apache.org/jira/browse/KAFKA-15197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17847366#comment-17847366 ]
Chia-Ping Tsai commented on KAFKA-15197: ---------------------------------------- It seems that behavior is expected. The check assumes that consumed offsets of "all" partitions gets updated after the consumer commits the offsets. However, the committed offsets are not the "end" offsets [0]. When the monotonic offsets of `OffsetSyncStore` is composed by end offset (upstream) only, we won't send any checkpoint as offsets translation won't generate the offset which is outside the bound [1]. For example: # tp has 500 records -> end offset is 500 # monotonic syncs is [499],[499],...[0],[0] # committed consumer offset is 449 # 499 (synced offset) is bigger than 449 (upstream group offset), and so we don't send checkpoint for it. In order to stabilize the tests, in short, we can change the condition from "all" to "some". For example, It works if 50% partitions get updated. Or we can tweak the translation algorithm to generate more smooth monotonic syncs [2]. This is more complicated and it can't solve the issue 100%. However, it could reduce of the possibility of updating nothing of group offset to downstream [0] [https://github.com/apache/kafka/blob/b8c96389b47df0dbd53fcba9404363dcdf43604d/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java#L716] [1] [https://github.com/apache/kafka/blob/b8c96389b47df0dbd53fcba9404363dcdf43604d/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java#L308] [2] [https://github.com/apache/kafka/blob/b8c96389b47df0dbd53fcba9404363dcdf43604d/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java#L212] [~gharris1727] WDYT? It would be great if MM2 expert can give me feedback :) > Flaky test > MirrorConnectorsIntegrationExactlyOnceTest.testOffsetTranslationBehindReplicationFlow() > -------------------------------------------------------------------------------------------------- > > Key: KAFKA-15197 > URL: https://issues.apache.org/jira/browse/KAFKA-15197 > Project: Kafka > Issue Type: Test > Components: mirrormaker > Reporter: Divij Vaidya > Priority: Major > Labels: flaky-test > Fix For: 3.8.0 > > > As of Jul 17th, this is the second most flaky test in our CI on trunk and > fails 46% of times. > See: > [https://ge.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.timeZoneId=Europe/Berlin] > > Note that MirrorConnectorsIntegrationExactlyOnceTest has multiple tests but > testOffsetTranslationBehindReplicationFlow is the one that is the reason for > most failures. see: > [https://ge.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.timeZoneId=Europe/Berlin&tests.container=org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest] > > > Reason for failure is: > |org.opentest4j.AssertionFailedError: Condition not met within timeout 20000. > Offsets for consumer group consumer-group-lagging-behind not translated from > primary for topic primary.test-topic-1 ==> expected: <true> but was: <false>| -- This message was sent by Atlassian Jira (v8.20.10#820010)