gharris1727 commented on code in PR #15910: URL: https://github.com/apache/kafka/pull/15910#discussion_r1605384701
########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java: ########## @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.KafkaBasedLog; +import org.apache.kafka.connect.util.TopicAdmin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.connect.mirror.MirrorCheckpointConfig.CHECKPOINTS_TARGET_CONSUMER_ROLE; + +/** + * Reads once the Kafka log for checkpoints and populates a map of + * checkpoints per consumer group. + * + * The Kafka log is closed after the initial load and only the in memory map is + * used after start. + */ +class CheckpointsStore implements AutoCloseable { Review Comment: optional nit: This class can be public, along with the methods that are intended to be used by MirrorCheckpointTask, because this isn't a publically-documented package (like clients, or connect-api, etc.) Outside of those publically-documented packages, the general practice is public for external callers, even if the current callers are in the same package. We only use package-local for things that would be protected/private, but need to be accessed in tests (and so come with the visibility comment.) ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java: ########## @@ -271,4 +284,102 @@ private Map<TopicPartition, Checkpoint> assertCheckpointForTopic( assertEquals(truth, checkpoints.containsKey(remoteTp), "should" + (truth ? "" : " not") + " emit offset sync"); return checkpoints; } + + @Test + public void testCheckpointsTaskRestartUsesExistingCheckpoints() { Review Comment: I think using "real checkpoints" generated by the first MirrorCheckpointTask to test the second MirrorCheckpointTask is not necessary, and you can use simulated checkpoints instead. Reassigning variables and copy-pasting sections in tests is typo-prone and I think we can avoid it here. ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java: ########## @@ -155,6 +154,58 @@ public void testPastOffsetTranslation() { } } + // this test has been wriiten knowing the exact offsets syncs stored + @Test + public void testPastOffsetTranslationWithoutInitializationReadToEnd() { + final int maxOffsetLag = 10; + + FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } + } + }; + + store.start(false); + + // After starting but before seeing new offsets + assertTranslationsNearby(store, 400, 480, 0); + assertTranslationsNearby(store, 500, 720, 480); + assertTranslationsNearby(store, 1000, 1000, 990); + + for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); + } + + // After seeing new offsets, 1000 was kicked out of the store, so + // 1000 can only be traslated to 1, only previously stored offset is 0 + assertTranslationsNearby(store, 1000, 3840, 0); + + // We can translate offsets between the latest startup offset and the latest offset with variable precision + // Older offsets are less precise and translation ends up farther apart + assertTranslationsNearby(store, 3840, 3840, 0); + assertTranslationsNearby(store, 7680, 7680, 3840); + assertTranslationsNearby(store, 8640, 8640, 7680); + assertTranslationsNearby(store, 9120, 9120, 8640); + assertTranslationsNearby(store, 9600, 9600, 9120); + assertTranslationsNearby(store, 9840, 9840, 9600); + assertTranslationsNearby(store, 9900, 9900, 9840); + assertTranslationsNearby(store, 9960, 9960, 9900); + assertTranslationsNearby(store, 9990, 9990, 9960); + assertTranslationsNearby(store, 10000, 10000, 9990); Review Comment: These are all expressible with assertSparseSync, there's no need for a new assert method. The offsets between the syncs (like 400, 500, and 1000) are implicitly part of the assertSparseSync calls. When testing `assertSparseSync(store, 480, 0)`, it asserts that 479 translates to 1, which also implies that 400 would translate to 1 as well. ```suggestion // After starting but before seeing new offsets assertSparseSync(store, 480, 0); assertSparseSync(store, 720, 480); assertSparseSync(store, 1000, 990); for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); } // After seeing new offsets, 1000 was kicked out of the store, so // offsets before 3840 can only be translated to 1, only previously stored offset is 0 assertSparseSync(store, 3840, 0); assertSparseSync(store, 7680, 3840); assertSparseSync(store, 8640, 7680); assertSparseSync(store, 9120, 8640); assertSparseSync(store, 9600, 9120); assertSparseSync(store, 9840, 9600); assertSparseSync(store, 9900, 9840); assertSparseSync(store, 9960, 9900); assertSparseSync(store, 9990, 9960); assertSparseSync(store, 10000, 9990); ``` ########## connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java: ########## @@ -119,7 +118,7 @@ public void testPastOffsetTranslation() { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); Review Comment: Can you move this into backingStoreStart, so that it happens when initializationMustReadToEnd is properly initialized? Here it's relying on the default value set by construction, not the value passed into `start`. Actually this seems to be the case in a lot of the tests here. Can you look through the tests, and whenever there are assertions or `sync` calls before `start`, apply the same fix? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
