From 66031ee69f61f05232aab6791fe9d8e98f2898c2 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Wed, 8 May 2024 16:03:39 +0100 Subject: [PATCH 01/15] KAFKA-15905 Restarts of MirrorCheckpointTask should not permanently interrupt offset translation MirrorCheckpointTask reloads the last checkpoint at start, OffsetSyncStore stores OffsetSyncs before reading till end. Add test case simulating restarted task where the store is reinitialized with later OffsetSyncs and check that emitted Checkpoint do not rewind. Also addresses KAFKA-16622 Mirromaker2 first Checkpoint not emitted until consumer group fully catches up once because the OffsetSyncStore store is populated before reading to log end. Co-Authored-By: Adrian Preston --- .../mirror/MirrorCheckpointConfig.java | 1 + .../connect/mirror/MirrorCheckpointTask.java | 44 +++++++++++++ .../kafka/connect/mirror/OffsetSyncStore.java | 2 +- .../mirror/MirrorCheckpointTaskTest.java | 64 +++++++++++++++++++ .../connect/mirror/OffsetSyncStoreTest.java | 62 +++++++++++------- 5 files changed, 148 insertions(+), 25 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java index e37cee4a79b8..8be52a9c9be9 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java @@ -75,6 +75,7 @@ public class MirrorCheckpointConfig extends MirrorConnectorConfig { public static final Class GROUP_FILTER_CLASS_DEFAULT = DefaultGroupFilter.class; public static final String OFFSET_SYNCS_SOURCE_CONSUMER_ROLE = "offset-syncs-source-consumer"; public static final String OFFSET_SYNCS_TARGET_CONSUMER_ROLE = "offset-syncs-target-consumer"; + public static final String CHECKPOINTS_TARGET_CONSUMER_ROLE = "checkpoints-target-consumer"; public static final String OFFSET_SYNCS_SOURCE_ADMIN_ROLE = "offset-syncs-source-admin"; public static final String OFFSET_SYNCS_TARGET_ADMIN_ROLE = "offset-syncs-target-admin"; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 96c287add5f1..8bff3a7f03bc 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -18,9 +18,11 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.data.Schema; @@ -29,6 +31,8 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.connect.util.KafkaBasedLog; +import org.apache.kafka.connect.util.TopicAdmin; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +51,7 @@ import java.time.Duration; import java.util.stream.Stream; +import static org.apache.kafka.connect.mirror.MirrorCheckpointConfig.CHECKPOINTS_TARGET_CONSUMER_ROLE; import static org.apache.kafka.connect.mirror.MirrorUtils.adminCall; /** Emits checkpoints for upstream consumer groups. */ @@ -70,6 +75,7 @@ public class MirrorCheckpointTask extends SourceTask { private Scheduler scheduler; private Map> idleConsumerGroupsOffset; private Map> checkpointsPerConsumerGroup; + public MirrorCheckpointTask() {} // for testing @@ -106,6 +112,7 @@ public void start(Map props) { checkpointsPerConsumerGroup = new HashMap<>(); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { + loadOldCheckpoints(config); offsetSyncStore.start(); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); @@ -116,6 +123,43 @@ public void start(Map props) { consumerGroups.size(), sourceClusterAlias, config.targetClusterAlias(), consumerGroups); } + private void loadOldCheckpoints(MirrorCheckpointTaskConfig config) { + TopicAdmin cpAdmin = null; + KafkaBasedLog oldCheckpoints = null; + + try { + cpAdmin = new TopicAdmin( + config.targetAdminConfig("checkpoint-target-admin"), + config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); + + oldCheckpoints = KafkaBasedLog.withExistingClients( + config.checkpointsTopic(), + MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), + null, + cpAdmin, + (error, cpRecord) -> this.handleCPRecord(cpRecord), + Time.SYSTEM, + ignored -> { + }, + topicPartition -> topicPartition.partition() == 0); + + oldCheckpoints.start(); + oldCheckpoints.stop(); + } finally { + Utils.closeQuietly(cpAdmin, "admin client for old CPs"); + Utils.closeQuietly(oldCheckpoints != null ? oldCheckpoints::stop : null, "backing store for old CPs"); + } + } + + private void handleCPRecord(ConsumerRecord cpRecord) { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpointsPerConsumerGroup.computeIfAbsent(cp.consumerGroupId(), ignored -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } + + @Override public void commit() { // nop diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index eca5cc68f997..726098c37188 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -214,7 +214,7 @@ private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSy // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all. - if (!readToEnd || syncs[0].upstreamOffset() > upstreamOffset) { + if (syncs[0].upstreamOffset() > upstreamOffset) { clearSyncArray(syncs, offsetSync); return; } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index c7aec3e54788..fc7624cbd88f 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -32,6 +32,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; public class MirrorCheckpointTaskTest { @@ -271,4 +272,67 @@ private Map assertCheckpointForTopic( assertEquals(truth, checkpoints.containsKey(remoteTp), "should" + (truth ? "" : " not") + " emit offset sync"); return checkpoints; } + + @Test + public void testCheckpointsTaskRestartUsesExistingCheckpoints() { + TopicPartition t1p0 = new TopicPartition("t1", 0); + TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); + offsetSyncStore.start(); + // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 + for (int i = 100; i <= 300; i += 100) { + offsetSyncStore.sync(t1p0, i, i); + } + + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + + // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 + // (as nearest mapping in OffsetSyncStore is 200->200) + Map upstreamGroupOffsets = new HashMap<>(); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + Map result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + + assertEquals(1, result.size()); + Checkpoint cp = result.get(sourceT1p0); + assertNotNull(cp); + assertEquals(250, cp.upstreamOffset()); + assertEquals(201, cp.downstreamOffset()); + + // Simulate task restart, during which more offsets are added to the sync topic, and thus the + // corresponding OffsetSyncStore no longer has a mapping for 100->100 + // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 + offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); + for (int i = 175; i <= 475; i += 100) { + offsetSyncStore.sync(t1p0, i, i); + } + offsetSyncStore.start(); + + // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) + Map> checkpointsPerConsumerGroup = new HashMap<>(); + checkpointsPerConsumerGroup.put("group1", result); + mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); + + // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is + // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. + assertEquals(OptionalLong.of(176), offsetSyncStore.translateDownstream(null, t1p0, 250)); + assertEquals(OptionalLong.of(176), offsetSyncStore.translateDownstream(null, t1p0, 370)); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(0, result.size()); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(370)); + result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(0, result.size()); + + // Upstream offset 400 has a closes downstream value of 376, and is emitted because it has + // a later downstream offset than the last checkpoint's downstream (201) + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(400)); + result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, result.size()); + cp = result.get(sourceT1p0); + assertNotNull(cp); + assertEquals(400, cp.upstreamOffset()); + assertEquals(376, cp.downstreamOffset()); + } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index bc76a1994db9..28d63060a4c6 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -110,6 +110,7 @@ public void testNoTranslationIfNoOffsetSync() { } } + // this test has been wriiten knowing the exact offsets syncs stored @Test public void testPastOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { @@ -117,38 +118,41 @@ public void testPastOffsetTranslation() { int offset = 0; for (; offset <= 1000; offset += maxOffsetLag) { store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + assertTranslationsNearbyInvariant(store, tp); } + store.start(); - // After starting but before seeing new offsets, only the latest startup offset can be translated - assertSparseSync(store, 1000, -1); + // After starting but before seeing new offsets + assertTranslationsNearby(store, 400, 480, 0); + assertTranslationsNearby(store, 500, 720, 480); + assertTranslationsNearby(store, 1000, 1000, 990); for (; offset <= 10000; offset += maxOffsetLag) { store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + assertTranslationsNearbyInvariant(store, tp); } - // After seeing new offsets, we still cannot translate earlier than the latest startup offset - // Invariant D: the last sync from the initial read-to-end is still stored - assertSparseSync(store, 1000, -1); + // 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 - assertSparseSync(store, 4840, 1000); - assertSparseSync(store, 6760, 4840); - assertSparseSync(store, 8680, 6760); - assertSparseSync(store, 9160, 8680); - assertSparseSync(store, 9640, 9160); - assertSparseSync(store, 9880, 9640); - assertSparseSync(store, 9940, 9880); - assertSparseSync(store, 9970, 9940); - assertSparseSync(store, 9990, 9970); - assertSparseSync(store, 10000, 9990); + 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); // Rewinding upstream offsets should clear all historical syncs store.sync(tp, 1500, 11000); - assertSparseSyncInvariant(store, tp); + assertTranslationsNearbyInvariant(store, tp); assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); @@ -239,11 +243,21 @@ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream } } - private void assertSparseSync(FakeOffsetSyncStore store, long syncOffset, long previousOffset) { - assertEquals(OptionalLong.of(previousOffset == -1 ? previousOffset : previousOffset + 1), store.translateDownstream(null, tp, syncOffset - 1)); - assertEquals(OptionalLong.of(syncOffset), store.translateDownstream(null, tp, syncOffset)); - assertEquals(OptionalLong.of(syncOffset + 1), store.translateDownstream(null, tp, syncOffset + 1)); - assertEquals(OptionalLong.of(syncOffset + 1), store.translateDownstream(null, tp, syncOffset + 2)); + private void assertTranslationsNearby(FakeOffsetSyncStore store, long syncOffsetToTranslate, long nearestLargeOrEqualsStoredOffset, long previousStoredOffset) { + // sanity test + assertTrue(syncOffsetToTranslate > previousStoredOffset); + + if (syncOffsetToTranslate == nearestLargeOrEqualsStoredOffset) { + assertEquals(OptionalLong.of(previousStoredOffset == -1 ? previousStoredOffset : previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate - 1)); + assertEquals(OptionalLong.of(syncOffsetToTranslate), store.translateDownstream(null, tp, syncOffsetToTranslate)); + assertEquals(OptionalLong.of(syncOffsetToTranslate + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 1)); + assertEquals(OptionalLong.of(syncOffsetToTranslate + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 2)); + } else if (syncOffsetToTranslate < nearestLargeOrEqualsStoredOffset) { + assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate - 1)); + assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate)); + assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 1)); + assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 2)); + } } private int countDistinctStoredSyncs(FakeOffsetSyncStore store, TopicPartition topicPartition) { @@ -256,7 +270,7 @@ private int countDistinctStoredSyncs(FakeOffsetSyncStore store, TopicPartition t return count; } - private void assertSparseSyncInvariant(FakeOffsetSyncStore store, TopicPartition topicPartition) { + private void assertTranslationsNearbyInvariant(FakeOffsetSyncStore store, TopicPartition topicPartition) { for (int j = 0; j < OffsetSyncStore.SYNCS_PER_PARTITION; j++) { for (int i = 0; i < j; i++) { long jUpstream = store.syncFor(topicPartition, j).upstreamOffset(); From 4ad350fb779e6ee44158ebb444d28ded7c7f887a Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Fri, 10 May 2024 10:50:16 +0100 Subject: [PATCH 02/15] MirrorCheckpointTask report errors when loading checkpoints Co-Authored-By: Adrioan Preston --- .../connect/mirror/MirrorCheckpointTask.java | 66 +++++++++++++------ .../kafka/connect/util/KafkaBasedLog.java | 12 +++- 2 files changed, 57 insertions(+), 21 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 8bff3a7f03bc..47c71b051cac 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -20,7 +20,9 @@ import org.apache.kafka.clients.admin.ConsumerGroupDescription; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.source.SourceTask; @@ -111,8 +113,8 @@ public void start(Map props) { idleConsumerGroupsOffset = new HashMap<>(); checkpointsPerConsumerGroup = new HashMap<>(); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); + this.initializeCheckpoints(config); scheduler.execute(() -> { - loadOldCheckpoints(config); offsetSyncStore.start(); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); @@ -123,43 +125,67 @@ public void start(Map props) { consumerGroups.size(), sourceClusterAlias, config.targetClusterAlias(), consumerGroups); } - private void loadOldCheckpoints(MirrorCheckpointTaskConfig config) { + // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state of this task + private void initializeCheckpoints(MirrorCheckpointTaskConfig config) { + + class CheckpointRecordHandler { + private volatile KafkaException lastLoggedErrorReadingCheckpoints = null; + + void handle(Throwable error, ConsumerRecord cpRecord) { + // See KafkaBasedLog.poll : only KafkaException can be passed as error + if (error instanceof KafkaException) { + // only log once + if ((lastLoggedErrorReadingCheckpoints == null || !lastLoggedErrorReadingCheckpoints.getClass().equals(error.getClass()))) { + log.error("Error loading Checkpoint topic", error); + lastLoggedErrorReadingCheckpoints = (KafkaException)error; + } + + if (error instanceof RetriableException) { + return; + } else { + throw (KafkaException) error; + } + } else { + lastLoggedErrorReadingCheckpoints = null; + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpointsPerConsumerGroup.computeIfAbsent(cp.consumerGroupId(), ignored -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } + } + } + + CheckpointRecordHandler handler = new CheckpointRecordHandler(); TopicAdmin cpAdmin = null; - KafkaBasedLog oldCheckpoints = null; + KafkaBasedLog previousCheckpoints = null; try { cpAdmin = new TopicAdmin( config.targetAdminConfig("checkpoint-target-admin"), config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); - oldCheckpoints = KafkaBasedLog.withExistingClients( + previousCheckpoints = KafkaBasedLog.withExistingClients( config.checkpointsTopic(), MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), null, cpAdmin, - (error, cpRecord) -> this.handleCPRecord(cpRecord), + (error, cpRecord) -> handler.handle(error, cpRecord), Time.SYSTEM, - ignored -> { - }, + ignored -> {}, topicPartition -> topicPartition.partition() == 0); - oldCheckpoints.start(); - oldCheckpoints.stop(); + log.info("Starting loading Checkpoint topic : {}", config.checkpointsTopic()); + previousCheckpoints.start(true); + previousCheckpoints.stop(); + log.info("Finished loading Checkpoint topic : {}", config.checkpointsTopic()); + log.debug("Initial checkpointsPerConsumerGroup : {}", checkpointsPerConsumerGroup); } finally { - Utils.closeQuietly(cpAdmin, "admin client for old CPs"); - Utils.closeQuietly(oldCheckpoints != null ? oldCheckpoints::stop : null, "backing store for old CPs"); + Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); + Utils.closeQuietly(previousCheckpoints != null ? previousCheckpoints::stop : null, "backing store for previous Checkpoints"); } } - private void handleCPRecord(ConsumerRecord cpRecord) { - Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); - if (consumerGroups.contains(cp.consumerGroupId())) { - Map cps = checkpointsPerConsumerGroup.computeIfAbsent(cp.consumerGroupId(), ignored -> new HashMap<>()); - cps.put(cp.topicPartition(), cp); - } - } - - @Override public void commit() { // nop diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index 36203399766c..cb44ca846b9d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -107,6 +107,9 @@ public class KafkaBasedLog { private boolean stopRequested; private final Queue> readLogEndOffsetCallbacks; private final java.util.function.Consumer initializer; + // initialized as false for backward compatibility + // TODO make the reportErrorsToCallback=true a default in Kafka 4.0 ? + private volatile boolean reportErrorsToCallback = false; /** * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until @@ -243,7 +246,11 @@ public void stop() { } public void start() { - log.info("Starting KafkaBasedLog with topic " + topic); + start(false); + } + public void start(boolean reportErrorsToCallback) { + this.reportErrorsToCallback = reportErrorsToCallback; + log.info("Starting KafkaBasedLog with topic {} reportErrorsToCallback={}", topic, reportErrorsToCallback); // Create the topic admin client and initialize the topic ... admin = topicAdminSupplier.get(); // may be null @@ -468,6 +475,9 @@ private void poll(long timeoutMs) { throw e; } catch (KafkaException e) { log.error("Error polling: " + e); + if (reportErrorsToCallback) { + consumedCallback.onCompletion(e, null); + } } } From 64a60b42619da095b2e688dc96a029f9175c2661 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Mon, 13 May 2024 17:18:20 +0100 Subject: [PATCH 03/15] If CheckpointTask cannot read checkpoints at startup, use previous OffsetSyncStore "pessimistic" load logic Co-Authored-By: Adrian Preston --- .../connect/mirror/MirrorCheckpointTask.java | 18 ++-- .../kafka/connect/mirror/OffsetSyncStore.java | 13 ++- .../mirror/MirrorCheckpointTaskTest.java | 15 ++-- .../connect/mirror/OffsetSyncStoreTest.java | 82 +++++++++++++++---- 4 files changed, 98 insertions(+), 30 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 47c71b051cac..ef01055701de 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -113,9 +113,9 @@ public void start(Map props) { idleConsumerGroupsOffset = new HashMap<>(); checkpointsPerConsumerGroup = new HashMap<>(); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); - this.initializeCheckpoints(config); scheduler.execute(() -> { - offsetSyncStore.start(); + boolean checkpointsRead = this.initializeCheckpoints(config); + offsetSyncStore.start(checkpointsRead); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), @@ -126,7 +126,7 @@ public void start(Map props) { } // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state of this task - private void initializeCheckpoints(MirrorCheckpointTaskConfig config) { + private boolean initializeCheckpoints(MirrorCheckpointTaskConfig config) { class CheckpointRecordHandler { private volatile KafkaException lastLoggedErrorReadingCheckpoints = null; @@ -135,9 +135,9 @@ void handle(Throwable error, ConsumerRecord cpRecord) { // See KafkaBasedLog.poll : only KafkaException can be passed as error if (error instanceof KafkaException) { // only log once - if ((lastLoggedErrorReadingCheckpoints == null || !lastLoggedErrorReadingCheckpoints.getClass().equals(error.getClass()))) { + if (lastLoggedErrorReadingCheckpoints == null || !lastLoggedErrorReadingCheckpoints.getClass().equals(error.getClass())) { log.error("Error loading Checkpoint topic", error); - lastLoggedErrorReadingCheckpoints = (KafkaException)error; + lastLoggedErrorReadingCheckpoints = (KafkaException) error; } if (error instanceof RetriableException) { @@ -145,7 +145,7 @@ void handle(Throwable error, ConsumerRecord cpRecord) { } else { throw (KafkaException) error; } - } else { + } else { // error is null lastLoggedErrorReadingCheckpoints = null; Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); if (consumerGroups.contains(cp.consumerGroupId())) { @@ -172,7 +172,7 @@ void handle(Throwable error, ConsumerRecord cpRecord) { cpAdmin, (error, cpRecord) -> handler.handle(error, cpRecord), Time.SYSTEM, - ignored -> {}, + ignored -> { }, topicPartition -> topicPartition.partition() == 0); log.info("Starting loading Checkpoint topic : {}", config.checkpointsTopic()); @@ -180,6 +180,10 @@ void handle(Throwable error, ConsumerRecord cpRecord) { previousCheckpoints.stop(); log.info("Finished loading Checkpoint topic : {}", config.checkpointsTopic()); log.debug("Initial checkpointsPerConsumerGroup : {}", checkpointsPerConsumerGroup); + return true; + } catch (KafkaException kexc) { + log.error("Failed to load previous checkpoints.", kexc); + return false; } finally { Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); Utils.closeQuietly(previousCheckpoints != null ? previousCheckpoints::stop : null, "backing store for previous Checkpoints"); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index 726098c37188..e646a2d3c9c0 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -63,6 +63,7 @@ class OffsetSyncStore implements AutoCloseable { private final KafkaBasedLog backingStore; private final Map offsetSyncs = new ConcurrentHashMap<>(); private final TopicAdmin admin; + protected volatile boolean pessimisticLoading = true; protected volatile boolean readToEnd = false; OffsetSyncStore(MirrorCheckpointConfig config) { @@ -104,8 +105,13 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig /** * Start the OffsetSyncStore, blocking until all previous Offset Syncs have been read from backing storage. + * @param optimisticLoading */ - public void start() { + public void start(boolean optimisticLoading) { + this.pessimisticLoading = !optimisticLoading; + if (pessimisticLoading) { + log.warn("OffsetSyncStore initialization will discard OffsetSyncs before end of topic"); + } backingStore.start(); readToEnd = true; } @@ -211,10 +217,13 @@ private void clearSyncArray(OffsetSync[] syncs, OffsetSync offsetSync) { private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSync offsetSync) { long upstreamOffset = offsetSync.upstreamOffset(); + // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all. - if (syncs[0].upstreamOffset() > upstreamOffset) { + boolean onlyLoadLastOffset = !readToEnd && pessimisticLoading; + boolean upstreamRewind = upstreamOffset < syncs[0].upstreamOffset(); + if (onlyLoadLastOffset || upstreamRewind) { clearSyncArray(syncs, offsetSync); return; } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index fc7624cbd88f..d1dfe0079495 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -59,7 +59,7 @@ public void testCheckpoint() { long t2UpstreamOffset = 7L; long t2DownstreamOffset = 8L; OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); @@ -201,7 +201,7 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu @Test public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); @@ -217,7 +217,7 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { @Test public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); @@ -228,7 +228,7 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { @Test public void testCheckpointRecordsMonotonicIfStoreRewinds() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.readToEnd(); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); @@ -278,12 +278,12 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { TopicPartition t1p0 = new TopicPartition("t1", 0); TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.setOptimisticLoading(); // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 for (int i = 100; i <= 300; i += 100) { offsetSyncStore.sync(t1p0, i, i); } - + offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); @@ -303,10 +303,11 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { // corresponding OffsetSyncStore no longer has a mapping for 100->100 // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); + offsetSyncStore.setOptimisticLoading(); for (int i = 175; i <= 475; i += 100) { offsetSyncStore.sync(t1p0, i, i); } - offsetSyncStore.start(); + offsetSyncStore.readToEnd(); // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) Map> checkpointsPerConsumerGroup = new HashMap<>(); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index 28d63060a4c6..250a2e5da686 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -39,10 +39,12 @@ static class FakeOffsetSyncStore extends OffsetSyncStore { super(); } - @Override - public void start() { - // do not call super to avoid NPE without a KafkaBasedLog. - readToEnd = true; + public void setOptimisticLoading() { + this.pessimisticLoading = false; + } + + public void readToEnd() { + this.readToEnd = true; } void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOffset) { @@ -57,7 +59,7 @@ void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOff @Test public void testOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.readToEnd(); // Emit synced downstream offset without dead-reckoning store.sync(tp, 100, 200); @@ -95,7 +97,7 @@ public void testNoTranslationIfStoreNotStarted() { assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); // After the store is started all offsets are visible - store.start(); + store.readToEnd(); assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); @@ -105,12 +107,11 @@ public void testNoTranslationIfStoreNotStarted() { @Test public void testNoTranslationIfNoOffsetSync() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.readToEnd(); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); } } - // this test has been wriiten knowing the exact offsets syncs stored @Test public void testPastOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { @@ -118,10 +119,56 @@ public void testPastOffsetTranslation() { int offset = 0; for (; offset <= 1000; offset += maxOffsetLag) { store.sync(tp, offset, offset); - assertTranslationsNearbyInvariant(store, tp); + assertSparseSyncInvariant(store, tp); + } + store.readToEnd(); + + // After starting but before seeing new offsets, only the latest startup offset can be translated + assertSparseSync(store, 1000, -1); + + for (; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); } - store.start(); + // After seeing new offsets, we still cannot translate earlier than the latest startup offset + // Invariant D: the last sync from the initial read-to-end is still stored + assertSparseSync(store, 1000, -1); + + // 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 + assertSparseSync(store, 4840, 1000); + assertSparseSync(store, 6760, 4840); + assertSparseSync(store, 8680, 6760); + assertSparseSync(store, 9160, 8680); + assertSparseSync(store, 9640, 9160); + assertSparseSync(store, 9880, 9640); + assertSparseSync(store, 9940, 9880); + assertSparseSync(store, 9970, 9940); + assertSparseSync(store, 9990, 9970); + assertSparseSync(store, 10000, 9990); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); + assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); + } + } + + // this test has been wriiten knowing the exact offsets syncs stored + @Test + public void testPastOffsetTranslationOptimisticLoading() { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { + int maxOffsetLag = 10; + int offset = 0; + store.setOptimisticLoading(); + for (; offset <= 1000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); + } + store.readToEnd(); // After starting but before seeing new offsets assertTranslationsNearby(store, 400, 480, 0); @@ -130,7 +177,7 @@ public void testPastOffsetTranslation() { for (; offset <= 10000; offset += maxOffsetLag) { store.sync(tp, offset, offset); - assertTranslationsNearbyInvariant(store, tp); + assertSparseSyncInvariant(store, tp); } // After seeing new offsets, 1000 was kicked out of the store, so @@ -152,7 +199,7 @@ public void testPastOffsetTranslation() { // Rewinding upstream offsets should clear all historical syncs store.sync(tp, 1500, 11000); - assertTranslationsNearbyInvariant(store, tp); + assertSparseSyncInvariant(store, tp); assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); @@ -219,7 +266,7 @@ public void testDroppedSyncsSpacing() { */ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream steps, int maximumExpirations) { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.readToEnd(); store.sync(tp, firstOffset, firstOffset); PrimitiveIterator.OfLong iterator = steps.iterator(); long offset = firstOffset; @@ -243,6 +290,13 @@ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream } } + private void assertSparseSync(FakeOffsetSyncStore store, long syncOffset, long previousOffset) { + assertEquals(OptionalLong.of(previousOffset == -1 ? previousOffset : previousOffset + 1), store.translateDownstream(null, tp, syncOffset - 1)); + assertEquals(OptionalLong.of(syncOffset), store.translateDownstream(null, tp, syncOffset)); + assertEquals(OptionalLong.of(syncOffset + 1), store.translateDownstream(null, tp, syncOffset + 1)); + assertEquals(OptionalLong.of(syncOffset + 1), store.translateDownstream(null, tp, syncOffset + 2)); + } + private void assertTranslationsNearby(FakeOffsetSyncStore store, long syncOffsetToTranslate, long nearestLargeOrEqualsStoredOffset, long previousStoredOffset) { // sanity test assertTrue(syncOffsetToTranslate > previousStoredOffset); @@ -270,7 +324,7 @@ private int countDistinctStoredSyncs(FakeOffsetSyncStore store, TopicPartition t return count; } - private void assertTranslationsNearbyInvariant(FakeOffsetSyncStore store, TopicPartition topicPartition) { + private void assertSparseSyncInvariant(FakeOffsetSyncStore store, TopicPartition topicPartition) { for (int j = 0; j < OffsetSyncStore.SYNCS_PER_PARTITION; j++) { for (int i = 0; i < j; i++) { long jUpstream = store.syncFor(topicPartition, j).upstreamOffset(); From e33edd2a728e99cc8e76f1cad0ff37557331138a Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Wed, 15 May 2024 14:14:21 +0100 Subject: [PATCH 04/15] Addressed review comments --- .../connect/mirror/MirrorCheckpointTask.java | 82 +++++++++---------- .../kafka/connect/mirror/OffsetSyncStore.java | 15 ++-- .../mirror/MirrorCheckpointTaskTest.java | 4 +- .../connect/mirror/OffsetSyncStoreTest.java | 8 +- 4 files changed, 51 insertions(+), 58 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index ef01055701de..a5370c46fa6f 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -18,12 +18,10 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ConsumerGroupDescription; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.ConsumerGroupState; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; -import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceRecord; @@ -47,6 +45,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.Collections; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.stream.Collectors; import java.util.concurrent.ExecutionException; @@ -111,11 +110,11 @@ public void start(Map props) { targetAdminClient = config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin")); metrics = config.metrics(); idleConsumerGroupsOffset = new HashMap<>(); - checkpointsPerConsumerGroup = new HashMap<>(); + Optional>> checkpoints = readCheckpoints(config); + checkpointsPerConsumerGroup = checkpoints.orElse(new HashMap<>()); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { - boolean checkpointsRead = this.initializeCheckpoints(config); - offsetSyncStore.start(checkpointsRead); + offsetSyncStore.start(!checkpoints.isPresent()); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), @@ -126,40 +125,15 @@ public void start(Map props) { } // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state of this task - private boolean initializeCheckpoints(MirrorCheckpointTaskConfig config) { - - class CheckpointRecordHandler { - private volatile KafkaException lastLoggedErrorReadingCheckpoints = null; - - void handle(Throwable error, ConsumerRecord cpRecord) { - // See KafkaBasedLog.poll : only KafkaException can be passed as error - if (error instanceof KafkaException) { - // only log once - if (lastLoggedErrorReadingCheckpoints == null || !lastLoggedErrorReadingCheckpoints.getClass().equals(error.getClass())) { - log.error("Error loading Checkpoint topic", error); - lastLoggedErrorReadingCheckpoints = (KafkaException) error; - } - - if (error instanceof RetriableException) { - return; - } else { - throw (KafkaException) error; - } - } else { // error is null - lastLoggedErrorReadingCheckpoints = null; - Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); - if (consumerGroups.contains(cp.consumerGroupId())) { - Map cps = checkpointsPerConsumerGroup.computeIfAbsent(cp.consumerGroupId(), ignored -> new HashMap<>()); - cps.put(cp.topicPartition(), cp); - } - } - } - } - - CheckpointRecordHandler handler = new CheckpointRecordHandler(); + // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog + // e.g. unauthorized to read from topic (non-retriable) + // if any are encountered, treat the loading of Checkpoints as failed. + private Optional>> readCheckpoints(MirrorCheckpointTaskConfig config) { + AtomicBoolean successful = new AtomicBoolean(true); TopicAdmin cpAdmin = null; KafkaBasedLog previousCheckpoints = null; + Map> checkpoints = new HashMap<>(); try { cpAdmin = new TopicAdmin( config.targetAdminConfig("checkpoint-target-admin"), @@ -170,20 +144,38 @@ void handle(Throwable error, ConsumerRecord cpRecord) { MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), null, cpAdmin, - (error, cpRecord) -> handler.handle(error, cpRecord), + (error, cpRecord) -> { + if (error != null && successful.getAndSet(false)) { + log.error("Error loading Checkpoint topic", error); + checkpoints.clear(); + } else if (successful.get()) { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + } + } + }, Time.SYSTEM, - ignored -> { }, + ignored -> { + }, topicPartition -> topicPartition.partition() == 0); log.info("Starting loading Checkpoint topic : {}", config.checkpointsTopic()); previousCheckpoints.start(true); previousCheckpoints.stop(); - log.info("Finished loading Checkpoint topic : {}", config.checkpointsTopic()); - log.debug("Initial checkpointsPerConsumerGroup : {}", checkpointsPerConsumerGroup); - return true; - } catch (KafkaException kexc) { - log.error("Failed to load previous checkpoints.", kexc); - return false; + if (successful.get()) { + log.info("Succesfully initialized checkpoints from topic : {}", config.checkpointsTopic()); + log.debug("Initial checkpointsPerConsumerGroup : {}", checkpoints); + return Optional.of(checkpoints); + } else { + log.warn("Failed initializing checkpoints from topic : {}", config.checkpointsTopic()); + return Optional.empty(); + } } finally { Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); Utils.closeQuietly(previousCheckpoints != null ? previousCheckpoints::stop : null, "backing store for previous Checkpoints"); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index e646a2d3c9c0..0c79da5a6e85 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -63,7 +63,7 @@ class OffsetSyncStore implements AutoCloseable { private final KafkaBasedLog backingStore; private final Map offsetSyncs = new ConcurrentHashMap<>(); private final TopicAdmin admin; - protected volatile boolean pessimisticLoading = true; + protected volatile boolean initializationMustReadToEnd = true; protected volatile boolean readToEnd = false; OffsetSyncStore(MirrorCheckpointConfig config) { @@ -105,12 +105,13 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig /** * Start the OffsetSyncStore, blocking until all previous Offset Syncs have been read from backing storage. - * @param optimisticLoading */ - public void start(boolean optimisticLoading) { - this.pessimisticLoading = !optimisticLoading; - if (pessimisticLoading) { - log.warn("OffsetSyncStore initialization will discard OffsetSyncs before end of topic"); + public void start(boolean initializationMustReadToEnd) { + this.initializationMustReadToEnd = initializationMustReadToEnd; + if (initializationMustReadToEnd) { + log.warn("OffsetSyncStore initializationMustReadToEnd = {}", initializationMustReadToEnd); + } else { + log.debug("OffsetSyncStore initializationMustReadToEnd = {}", initializationMustReadToEnd); } backingStore.start(); readToEnd = true; @@ -221,7 +222,7 @@ private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSy // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all. - boolean onlyLoadLastOffset = !readToEnd && pessimisticLoading; + boolean onlyLoadLastOffset = !readToEnd && initializationMustReadToEnd; boolean upstreamRewind = upstreamOffset < syncs[0].upstreamOffset(); if (onlyLoadLastOffset || upstreamRewind) { clearSyncArray(syncs, offsetSync); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index d1dfe0079495..6a3e46d3f3b0 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -278,7 +278,7 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { TopicPartition t1p0 = new TopicPartition("t1", 0); TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.setOptimisticLoading(); + offsetSyncStore.setLoadingMustReadToEnd(false); // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 for (int i = 100; i <= 300; i += 100) { offsetSyncStore.sync(t1p0, i, i); @@ -303,7 +303,7 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { // corresponding OffsetSyncStore no longer has a mapping for 100->100 // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.setOptimisticLoading(); + offsetSyncStore.setLoadingMustReadToEnd(false); for (int i = 175; i <= 475; i += 100) { offsetSyncStore.sync(t1p0, i, i); } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index 250a2e5da686..a3eeee270f5b 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -39,8 +39,8 @@ static class FakeOffsetSyncStore extends OffsetSyncStore { super(); } - public void setOptimisticLoading() { - this.pessimisticLoading = false; + public void setLoadingMustReadToEnd(boolean b) { + this.initializationMustReadToEnd = false; } public void readToEnd() { @@ -159,11 +159,11 @@ public void testPastOffsetTranslation() { // this test has been wriiten knowing the exact offsets syncs stored @Test - public void testPastOffsetTranslationOptimisticLoading() { + public void testPastOffsetTranslationWithoutInitializationReadToEnd() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { int maxOffsetLag = 10; int offset = 0; - store.setOptimisticLoading(); + store.setLoadingMustReadToEnd(false); for (; offset <= 1000; offset += maxOffsetLag) { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); From 2146ed0918020f4a67277a1b97108cae835ac6bc Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Wed, 15 May 2024 16:31:06 +0100 Subject: [PATCH 05/15] Unit test for MirrorCheckpointTask.readCheckpoints --- .../kafka/connect/mirror/Checkpoint.java | 14 ++++ .../connect/mirror/MirrorCheckpointTask.java | 70 +++++++++------- .../mirror/MirrorCheckpointTaskTest.java | 84 +++++++++++++++++-- 3 files changed, 130 insertions(+), 38 deletions(-) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java index 8f186400dd29..353d2eedb959 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.HashMap; import java.nio.ByteBuffer; +import java.util.Objects; /** Checkpoint records emitted from MirrorCheckpointConnector. Encodes remote consumer group state. */ public class Checkpoint { @@ -180,5 +181,18 @@ byte[] recordKey() { byte[] recordValue() { return serializeValue(VERSION).array(); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Checkpoint that = (Checkpoint) o; + return upstreamOffset == that.upstreamOffset && downstreamOffset == that.downstreamOffset && Objects.equals(consumerGroupId, that.consumerGroupId) && Objects.equals(topicPartition, that.topicPartition) && Objects.equals(metadata, that.metadata); + } + + @Override + public int hashCode() { + return Objects.hash(consumerGroupId, topicPartition, upstreamOffset, downstreamOffset, metadata); + } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index a5370c46fa6f..3a3eb3eb1436 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.errors.UnknownMemberIdException; @@ -31,6 +32,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.RecordMetadata; +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; @@ -81,13 +83,14 @@ public MirrorCheckpointTask() {} // for testing MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, - ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, Map> idleConsumerGroupsOffset, Map> checkpointsPerConsumerGroup) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; this.offsetSyncStore = offsetSyncStore; + this.consumerGroups = consumerGroups; this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; this.topicFilter = topic -> true; @@ -128,12 +131,45 @@ public void start(Map props) { // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog // e.g. unauthorized to read from topic (non-retriable) // if any are encountered, treat the loading of Checkpoints as failed. - private Optional>> readCheckpoints(MirrorCheckpointTaskConfig config) { + Optional>> readCheckpoints(MirrorCheckpointTaskConfig config) { AtomicBoolean successful = new AtomicBoolean(true); + Map> checkpoints = new HashMap<>(); + Callback> consumedCallback = new Callback>() { + @Override + public void onCompletion(Throwable error, ConsumerRecord cpRecord) { + if (error != null && successful.getAndSet(false)) { + log.error("Error loading Checkpoint topic", error); + checkpoints.clear(); + } else if (successful.get()) { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + } + } + } + }; + + log.info("Starting loading Checkpoint topic : {}", config.checkpointsTopic()); + readCheckpointsImpl(config, consumedCallback); + if (successful.get()) { + log.info("Succesfully initialized checkpoints from topic : {}", config.checkpointsTopic()); + log.debug("Initial checkpointsPerConsumerGroup : {}", checkpoints); + return Optional.of(checkpoints); + } else { + log.warn("Failed initializing checkpoints from topic : {}", config.checkpointsTopic()); + return Optional.empty(); + } + } + + // accessible for testing + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { TopicAdmin cpAdmin = null; KafkaBasedLog previousCheckpoints = null; - - Map> checkpoints = new HashMap<>(); try { cpAdmin = new TopicAdmin( config.targetAdminConfig("checkpoint-target-admin"), @@ -144,38 +180,14 @@ private Optional>> readCheckpoints(M MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), null, cpAdmin, - (error, cpRecord) -> { - if (error != null && successful.getAndSet(false)) { - log.error("Error loading Checkpoint topic", error); - checkpoints.clear(); - } else if (successful.get()) { - try { - Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); - if (consumerGroups.contains(cp.consumerGroupId())) { - Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); - cps.put(cp.topicPartition(), cp); - } - } catch (SchemaException ex) { - log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); - } - } - }, + consumedCallback, Time.SYSTEM, ignored -> { }, topicPartition -> topicPartition.partition() == 0); - log.info("Starting loading Checkpoint topic : {}", config.checkpointsTopic()); previousCheckpoints.start(true); previousCheckpoints.stop(); - if (successful.get()) { - log.info("Succesfully initialized checkpoints from topic : {}", config.checkpointsTopic()); - log.debug("Initial checkpointsPerConsumerGroup : {}", checkpoints); - return Optional.of(checkpoints); - } else { - log.warn("Failed initializing checkpoints from topic : {}", config.checkpointsTopic()); - return Optional.empty(); - } } finally { Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); Utils.closeQuietly(previousCheckpoints != null ? previousCheckpoints::stop : null, "backing store for previous Checkpoints"); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index 6a3e46d3f3b0..9c37d2bfbe11 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -17,16 +17,21 @@ package org.apache.kafka.connect.mirror; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Collections; import java.util.Optional; import java.util.OptionalLong; +import java.util.Set; import java.util.concurrent.ExecutionException; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.util.Callback; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -34,13 +39,15 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class MirrorCheckpointTaskTest { @Test public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -61,7 +68,7 @@ public void testCheckpoint() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -161,7 +168,7 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { checkpointsPerConsumerGroup.put(consumer2, checkpointMapC2); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, checkpointsPerConsumerGroup); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -191,7 +198,7 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu checkpointsPerConsumerGroup.put(consumer, checkpointMap); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source", "target", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, checkpointsPerConsumerGroup); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -203,7 +210,7 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); Optional checkpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 1), @@ -219,7 +226,7 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -231,7 +238,7 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { offsetSyncStore.readToEnd(); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), checkpointsPerConsumerGroup); TopicPartition tp = new TopicPartition("topic1", 0); TopicPartition targetTP = new TopicPartition("source1.topic1", 0); @@ -285,7 +292,7 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { } offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 // (as nearest mapping in OffsetSyncStore is 200->200) @@ -313,7 +320,7 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { Map> checkpointsPerConsumerGroup = new HashMap<>(); checkpointsPerConsumerGroup.put("group1", result); mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), checkpointsPerConsumerGroup); // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. @@ -336,4 +343,63 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { assertEquals(400, cp.upstreamOffset()); assertEquals(376, cp.downstreamOffset()); } + + @Test + public void testReadCheckpointsTopic() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target1", + new DefaultReplicationPolicy(), null, + consumerGroups, Collections.emptyMap(), Collections.emptyMap()) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + // this record must be ignored as not part of consumerGroups for task + consumedCallback.onCompletion(null, newCheckpointRecord("group2", "t1", 0, 0, 0)); + // this record must be ignored as malformed + consumedCallback.onCompletion(null, + new ConsumerRecord<>("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); + } + }; + + Optional>> checkpoints = mirrorCheckpointTask.readCheckpoints(config); + assertTrue(checkpoints.isPresent()); + + Map> expected = new HashMap<>(); + expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), + new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); + assertEquals(expected, checkpoints.get()); + } + + @Test + public void testReadCheckpointsTopicError() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target1", + new DefaultReplicationPolicy(), null, + consumerGroups, Collections.emptyMap(), Collections.emptyMap()) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + consumedCallback.onCompletion(new KafkaException("test"), null); + } + }; + + Optional>> checkpoints = mirrorCheckpointTask.readCheckpoints(config); + assertFalse(checkpoints.isPresent()); + } + + ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { + Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); + return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); + } } From 7115d89ffee5c79d6aa217fb6f85f0024842d95c Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Thu, 16 May 2024 12:41:54 +0100 Subject: [PATCH 06/15] Load checkpoints asynchronously. Use a wrapper to the checkpointsPerGroup map to ensure thread safety on restart. The callback rethrows the error as a way to stop the KafkaBasedLog looping forever in readtoEnd. This occurs when unauthorized to READ from the topic, as KafkaBasedLog retries indefinitely. Unauthorized to DESCRIBE instead fails immediately before fetching. All such failures result in a warning log message about degraded offset translation. --- .../connect/mirror/CheckpointsStore.java | 173 ++++++++++++++++++ .../connect/mirror/MirrorCheckpointTask.java | 106 ++--------- .../kafka/connect/mirror/OffsetSyncStore.java | 7 +- .../connect/mirror/CheckpointsStoreTest.java | 100 ++++++++++ .../mirror/MirrorCheckpointTaskTest.java | 93 ++-------- 5 files changed, 310 insertions(+), 169 deletions(-) create mode 100644 connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java new file mode 100644 index 000000000000..5943a473301a --- /dev/null +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java @@ -0,0 +1,173 @@ +/* + * 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.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.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 + */ +public class CheckpointsStore implements AutoCloseable { + + private static final Logger log = LoggerFactory.getLogger(CheckpointsStore.class); + + private final MirrorCheckpointTaskConfig config; + private final Set consumerGroups; + + private TopicAdmin cpAdmin = null; + private KafkaBasedLog backingStore = null; + private Map> checkpointsPerConsumerGroup; + + private volatile boolean loadSuccess = false; + private volatile boolean isInitialized = false; + + public CheckpointsStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + this.config = config; + this.consumerGroups = new HashSet<>(consumerGroups); + } + + // for testing + CheckpointsStore(Map> checkpointsPerConsumerGroup) { + this.config = null; + this.consumerGroups = null; + this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + isInitialized = true; + loadSuccess = true; + } + + // potentially long running + public void start() { + checkpointsPerConsumerGroup = readCheckpoints(); + isInitialized = true; + log.trace("Checkpoints store content : {}", checkpointsPerConsumerGroup); + } + + public boolean loadSuccess() { + return loadSuccess; + } + + public boolean isInitialized() { + return isInitialized; + } + + + // return a mutable map - it is expected to be mutated by the Task + public Map> contents() { + return checkpointsPerConsumerGroup; + } + + @Override + public void close() { + Utils.closeQuietly(backingStore != null ? backingStore::stop : null, "backing store for previous Checkpoints"); + Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); + cpAdmin = null; + backingStore = null; + } + + // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state + // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog + // e.g. unauthorized to read from topic (non-retriable) + // if any are encountered, treat the loading of Checkpoints as failed. + public Map> readCheckpoints() { + Map> checkpoints = new HashMap<>(); + Callback> consumedCallback = new Callback>() { + @Override + public void onCompletion(Throwable error, ConsumerRecord cpRecord) { + if (error != null) { + // if there is no authorization to READ from the topic, we must throw an error + // to stop the KafkaBasedLog forever looping attempting to read to end + checkpoints.clear(); + if (error instanceof RuntimeException) { + throw (RuntimeException) error; + } else { + throw new RuntimeException(error); + } + } else { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + } + } + } + }; + + long startTime = System.currentTimeMillis(); + try { + readCheckpointsImpl(config, consumedCallback); + log.info("Loading Checkpoints topic took {}ms", System.currentTimeMillis() - startTime); + loadSuccess = true; + } catch (Exception error) { + loadSuccess = false; + if (error instanceof AuthorizationException) { + log.warn("Not authorized to access checkpoints topic {} - this will degrade offset translation as fewer checkpoints may be emitted", config.checkpointsTopic(), error); + } else { + log.info("Exception encountered loading Checkpoint topic {} - this will degrade offset translation as fewer checkpoints may be emitted", config.checkpointsTopic(), error); + } + } + return checkpoints; + } + + // accessible for testing + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + try { + cpAdmin = new TopicAdmin( + config.targetAdminConfig("checkpoint-target-admin"), + config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); + + backingStore = KafkaBasedLog.withExistingClients( + config.checkpointsTopic(), + MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), + null, + cpAdmin, + consumedCallback, + Time.SYSTEM, + ignored -> { + }, + topicPartition -> topicPartition.partition() == 0); + + backingStore.start(true); + backingStore.stop(); + } finally { + // closing early to free resources + close(); + } + } + +} diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 3a3eb3eb1436..89cfcf992ea4 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -18,12 +18,9 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ConsumerGroupDescription; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.errors.UnknownMemberIdException; -import org.apache.kafka.common.protocol.types.SchemaException; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.data.Schema; @@ -32,9 +29,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.RecordMetadata; -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; @@ -47,14 +41,12 @@ import java.util.OptionalLong; import java.util.Set; import java.util.Collections; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.stream.Collectors; import java.util.concurrent.ExecutionException; import java.time.Duration; import java.util.stream.Stream; -import static org.apache.kafka.connect.mirror.MirrorCheckpointConfig.CHECKPOINTS_TARGET_CONSUMER_ROLE; import static org.apache.kafka.connect.mirror.MirrorUtils.adminCall; /** Emits checkpoints for upstream consumer groups. */ @@ -77,22 +69,22 @@ public class MirrorCheckpointTask extends SourceTask { private MirrorCheckpointMetrics metrics; private Scheduler scheduler; private Map> idleConsumerGroupsOffset; - private Map> checkpointsPerConsumerGroup; + private CheckpointsStore checkpointsStore; public MirrorCheckpointTask() {} // for testing MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, - ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, - Map> idleConsumerGroupsOffset, - Map> checkpointsPerConsumerGroup) { + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, + Map> idleConsumerGroupsOffset, + CheckpointsStore checkpointsStore) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; this.offsetSyncStore = offsetSyncStore; this.consumerGroups = consumerGroups; this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; - this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + this.checkpointsStore = checkpointsStore; this.topicFilter = topic -> true; } @@ -113,87 +105,22 @@ public void start(Map props) { targetAdminClient = config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin")); metrics = config.metrics(); idleConsumerGroupsOffset = new HashMap<>(); - Optional>> checkpoints = readCheckpoints(config); - checkpointsPerConsumerGroup = checkpoints.orElse(new HashMap<>()); + checkpointsStore = new CheckpointsStore(config, consumerGroups); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { - offsetSyncStore.start(!checkpoints.isPresent()); + // loading the stores are potentially long running operations, so they run asynchronously + // to avoid blocking task::start (until a task has completed starting it cannot be stopped) + checkpointsStore.start(); + offsetSyncStore.start(!checkpointsStore.loadSuccess()); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), "sync idle consumer group offset from source to target"); - }, "starting offset sync store"); + }, "starting checkpoint and offset sync stores"); log.info("{} checkpointing {} consumer groups {}->{}: {}.", Thread.currentThread().getName(), consumerGroups.size(), sourceClusterAlias, config.targetClusterAlias(), consumerGroups); } - // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state of this task - // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog - // e.g. unauthorized to read from topic (non-retriable) - // if any are encountered, treat the loading of Checkpoints as failed. - Optional>> readCheckpoints(MirrorCheckpointTaskConfig config) { - AtomicBoolean successful = new AtomicBoolean(true); - Map> checkpoints = new HashMap<>(); - Callback> consumedCallback = new Callback>() { - @Override - public void onCompletion(Throwable error, ConsumerRecord cpRecord) { - if (error != null && successful.getAndSet(false)) { - log.error("Error loading Checkpoint topic", error); - checkpoints.clear(); - } else if (successful.get()) { - try { - Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); - if (consumerGroups.contains(cp.consumerGroupId())) { - Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); - cps.put(cp.topicPartition(), cp); - } - } catch (SchemaException ex) { - log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); - } - } - } - }; - - log.info("Starting loading Checkpoint topic : {}", config.checkpointsTopic()); - readCheckpointsImpl(config, consumedCallback); - if (successful.get()) { - log.info("Succesfully initialized checkpoints from topic : {}", config.checkpointsTopic()); - log.debug("Initial checkpointsPerConsumerGroup : {}", checkpoints); - return Optional.of(checkpoints); - } else { - log.warn("Failed initializing checkpoints from topic : {}", config.checkpointsTopic()); - return Optional.empty(); - } - } - - // accessible for testing - void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { - TopicAdmin cpAdmin = null; - KafkaBasedLog previousCheckpoints = null; - try { - cpAdmin = new TopicAdmin( - config.targetAdminConfig("checkpoint-target-admin"), - config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); - - previousCheckpoints = KafkaBasedLog.withExistingClients( - config.checkpointsTopic(), - MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), - null, - cpAdmin, - consumedCallback, - Time.SYSTEM, - ignored -> { - }, - topicPartition -> topicPartition.partition() == 0); - - previousCheckpoints.start(true); - previousCheckpoints.stop(); - } finally { - Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); - Utils.closeQuietly(previousCheckpoints != null ? previousCheckpoints::stop : null, "backing store for previous Checkpoints"); - } - } - @Override public void commit() { // nop @@ -204,6 +131,7 @@ public void stop() { long start = System.currentTimeMillis(); stopping = true; Utils.closeQuietly(topicFilter, "topic filter"); + Utils.closeQuietly(checkpointsStore, "checkpoints store"); Utils.closeQuietly(offsetSyncStore, "offset sync store"); Utils.closeQuietly(sourceAdminClient, "source admin client"); Utils.closeQuietly(targetAdminClient, "target admin client"); @@ -224,8 +152,8 @@ public List poll() throws InterruptedException { while (!stopping && System.currentTimeMillis() < deadline) { Thread.sleep(pollTimeout.toMillis()); } - if (stopping) { - // we are stopping, return early. + if (stopping || !checkpointsStore.isInitialized()) { + // we are stopping, or not fully initialized, return early. return null; } List records = new ArrayList<>(); @@ -250,7 +178,7 @@ private List sourceRecordsForGroup(String group) throws Interrupte long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); Map newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); + Map oldCheckpoints = checkpointsStore.contents().computeIfAbsent(group, ignored -> new HashMap<>()); oldCheckpoints.putAll(newCheckpoints); return newCheckpoints.values().stream() .map(x -> checkpointRecord(x, timestamp)) @@ -273,7 +201,7 @@ Map checkpointsForGroup(Map checkpoints = checkpointsPerConsumerGroup.get(checkpoint.consumerGroupId()); + Map checkpoints = checkpointsStore.contents().get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; @@ -473,7 +401,7 @@ void syncGroupOffset(String consumerGroupId, Map> getConvertedUpstreamOffset() { Map> result = new HashMap<>(); - for (Entry> entry : checkpointsPerConsumerGroup.entrySet()) { + for (Entry> entry : checkpointsStore.contents().entrySet()) { String consumerId = entry.getKey(); Map convertedUpstreamOffset = new HashMap<>(); for (Checkpoint checkpoint : entry.getValue().values()) { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index 0c79da5a6e85..a0dbde48fc68 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -108,11 +108,8 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig */ public void start(boolean initializationMustReadToEnd) { this.initializationMustReadToEnd = initializationMustReadToEnd; - if (initializationMustReadToEnd) { - log.warn("OffsetSyncStore initializationMustReadToEnd = {}", initializationMustReadToEnd); - } else { - log.debug("OffsetSyncStore initializationMustReadToEnd = {}", initializationMustReadToEnd); - } + log.info("OffsetSyncStore initializationMustReadToEnd:{}{}", initializationMustReadToEnd, + initializationMustReadToEnd ? " - fewer checkpoints may be emitted" : ""); backingStore.start(); readToEnd = true; } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java new file mode 100644 index 000000000000..7cd0c1aa2884 --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java @@ -0,0 +1,100 @@ +/* + * 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.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.connect.util.Callback; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CheckpointsStoreTest { + + @Test + public void testReadCheckpointsTopic() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + CheckpointsStore store = new CheckpointsStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + // this record must be ignored as not part of consumerGroups for task + consumedCallback.onCompletion(null, newCheckpointRecord("group2", "t1", 0, 0, 0)); + // this record must be ignored as malformed + consumedCallback.onCompletion(null, + new ConsumerRecord<>("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); + } + }; + assertFalse(store.loadSuccess()); + assertFalse(store.isInitialized()); + + store.start(); + + assertTrue(store.loadSuccess()); + assertTrue(store.isInitialized()); + + Map> expected = new HashMap<>(); + expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), + new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); + assertEquals(expected, store.contents()); + } + + @Test + public void testReadCheckpointsTopicError() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + CheckpointsStore store = new CheckpointsStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + consumedCallback.onCompletion(new TopicAuthorizationException("test"), null); + } + }; + + store.start(); + assertFalse(store.loadSuccess()); + assertTrue(store.isInitialized()); + assertTrue(store.contents().isEmpty()); + } + + ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { + Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); + return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); + } + +} diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index 9c37d2bfbe11..d2f9029f9f92 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -17,21 +17,16 @@ package org.apache.kafka.connect.mirror; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Collections; import java.util.Optional; import java.util.OptionalLong; -import java.util.Set; import java.util.concurrent.ExecutionException; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.util.Callback; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -39,15 +34,14 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class MirrorCheckpointTaskTest { @Test public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(Collections.emptyMap())); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -68,7 +62,8 @@ public void testCheckpoint() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -168,7 +163,8 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { checkpointsPerConsumerGroup.put(consumer2, checkpointMapC2); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointsStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -198,7 +194,8 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu checkpointsPerConsumerGroup.put(consumer, checkpointMap); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source", "target", - new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointsStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -210,7 +207,8 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); Optional checkpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 1), @@ -226,7 +224,8 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -238,7 +237,8 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { offsetSyncStore.readToEnd(); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(checkpointsPerConsumerGroup)); TopicPartition tp = new TopicPartition("topic1", 0); TopicPartition targetTP = new TopicPartition("source1.topic1", 0); @@ -292,7 +292,8 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { } offsetSyncStore.readToEnd(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(Collections.emptyMap())); // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 // (as nearest mapping in OffsetSyncStore is 200->200) @@ -320,7 +321,8 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { Map> checkpointsPerConsumerGroup = new HashMap<>(); checkpointsPerConsumerGroup.put("group1", result); mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointsStore(checkpointsPerConsumerGroup)); // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. @@ -343,63 +345,4 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { assertEquals(400, cp.upstreamOffset()); assertEquals(376, cp.downstreamOffset()); } - - @Test - public void testReadCheckpointsTopic() { - Set consumerGroups = new HashSet<>(); - consumerGroups.add("group1"); - - MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); - when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); - - MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target1", - new DefaultReplicationPolicy(), null, - consumerGroups, Collections.emptyMap(), Collections.emptyMap()) { - @Override - void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { - consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); - // this record must be ignored as not part of consumerGroups for task - consumedCallback.onCompletion(null, newCheckpointRecord("group2", "t1", 0, 0, 0)); - // this record must be ignored as malformed - consumedCallback.onCompletion(null, - new ConsumerRecord<>("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); - consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); - } - }; - - Optional>> checkpoints = mirrorCheckpointTask.readCheckpoints(config); - assertTrue(checkpoints.isPresent()); - - Map> expected = new HashMap<>(); - expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), - new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); - assertEquals(expected, checkpoints.get()); - } - - @Test - public void testReadCheckpointsTopicError() { - Set consumerGroups = new HashSet<>(); - consumerGroups.add("group1"); - - MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); - when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); - - MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target1", - new DefaultReplicationPolicy(), null, - consumerGroups, Collections.emptyMap(), Collections.emptyMap()) { - @Override - void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { - consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); - consumedCallback.onCompletion(new KafkaException("test"), null); - } - }; - - Optional>> checkpoints = mirrorCheckpointTask.readCheckpoints(config); - assertFalse(checkpoints.isPresent()); - } - - ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { - Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); - return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); - } } From cc27e4c01d20ae6b705ad19a51b089f804dd370a Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Fri, 17 May 2024 11:31:14 +0100 Subject: [PATCH 07/15] Addressed review comments --- .../connect/mirror/CheckpointsStore.java | 114 +++++++++++------- .../connect/mirror/MirrorCheckpointTask.java | 31 ++--- .../kafka/connect/mirror/OffsetSyncStore.java | 10 +- .../connect/mirror/CheckpointsStoreTest.java | 13 +- .../mirror/MirrorCheckpointTaskTest.java | 51 ++++---- .../connect/mirror/OffsetSyncStoreTest.java | 98 +++++++-------- .../kafka/connect/util/KafkaBasedLog.java | 2 +- 7 files changed, 168 insertions(+), 151 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java index 5943a473301a..865135e26bc6 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java @@ -17,6 +17,7 @@ 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; @@ -28,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -37,9 +39,12 @@ /** * Reads once the Kafka log for checkpoints and populates a map of - * checkpoints per consumer group + * checkpoints per consumer group. + * + * The Kafka log is closed after the initial load and only the in memory map is + * used after start. */ -public class CheckpointsStore implements AutoCloseable { +class CheckpointsStore implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(CheckpointsStore.class); @@ -48,48 +53,68 @@ public class CheckpointsStore implements AutoCloseable { private TopicAdmin cpAdmin = null; private KafkaBasedLog backingStore = null; - private Map> checkpointsPerConsumerGroup; + // accessible for testing + Map> checkpointsPerConsumerGroup; private volatile boolean loadSuccess = false; private volatile boolean isInitialized = false; - public CheckpointsStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + CheckpointsStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { this.config = config; this.consumerGroups = new HashSet<>(consumerGroups); } - // for testing + // constructor for testing only CheckpointsStore(Map> checkpointsPerConsumerGroup) { - this.config = null; - this.consumerGroups = null; + this.config = null; //ignored by tests + this.consumerGroups = null; //ignored by tests this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; isInitialized = true; loadSuccess = true; } // potentially long running - public void start() { + boolean start() { checkpointsPerConsumerGroup = readCheckpoints(); isInitialized = true; - log.trace("Checkpoints store content : {}", checkpointsPerConsumerGroup); - } - - public boolean loadSuccess() { + log.trace("CheckpointsStore started, load success={}, map={}", loadSuccess, checkpointsPerConsumerGroup); return loadSuccess; } - public boolean isInitialized() { + boolean isInitialized() { return isInitialized; } + void update(String group, Map newCheckpoints) { + Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); + oldCheckpoints.putAll(newCheckpoints); + } + + Map get(String group) { + Map result = checkpointsPerConsumerGroup.get(group); + return result == null ? null : Collections.unmodifiableMap(result); + } + + Map> computeConvertedUpstreamOffset() { + Map> result = new HashMap<>(); - // return a mutable map - it is expected to be mutated by the Task - public Map> contents() { - return checkpointsPerConsumerGroup; + for (Map.Entry> entry : checkpointsPerConsumerGroup.entrySet()) { + String consumerId = entry.getKey(); + Map convertedUpstreamOffset = new HashMap<>(); + for (Checkpoint checkpoint : entry.getValue().values()) { + convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); + } + result.put(consumerId, convertedUpstreamOffset); + } + return result; } @Override public void close() { + releaseResources(); + } + + private void releaseResources() { Utils.closeQuietly(backingStore != null ? backingStore::stop : null, "backing store for previous Checkpoints"); Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); cpAdmin = null; @@ -100,45 +125,48 @@ public void close() { // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog // e.g. unauthorized to read from topic (non-retriable) // if any are encountered, treat the loading of Checkpoints as failed. - public Map> readCheckpoints() { + Map> readCheckpoints() { Map> checkpoints = new HashMap<>(); - Callback> consumedCallback = new Callback>() { - @Override - public void onCompletion(Throwable error, ConsumerRecord cpRecord) { - if (error != null) { - // if there is no authorization to READ from the topic, we must throw an error - // to stop the KafkaBasedLog forever looping attempting to read to end - checkpoints.clear(); - if (error instanceof RuntimeException) { - throw (RuntimeException) error; - } else { - throw new RuntimeException(error); - } + Callback> consumedCallback = (error, cpRecord) -> { + if (error != null) { + // if there is no authorization to READ from the topic, we must throw an error + // to stop the KafkaBasedLog forever looping attempting to read to end + checkpoints.clear(); + if (error instanceof RuntimeException) { + throw (RuntimeException) error; } else { - try { - Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); - if (consumerGroups.contains(cp.consumerGroupId())) { - Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); - cps.put(cp.topicPartition(), cp); - } - } catch (SchemaException ex) { - log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + throw new RuntimeException(error); + } + } else { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); } } }; - long startTime = System.currentTimeMillis(); try { + long startTime = System.currentTimeMillis(); readCheckpointsImpl(config, consumedCallback); - log.info("Loading Checkpoints topic took {}ms", System.currentTimeMillis() - startTime); + log.debug("starting+stopping KafkaBasedLog took {}ms", System.currentTimeMillis() - startTime); loadSuccess = true; } catch (Exception error) { loadSuccess = false; if (error instanceof AuthorizationException) { - log.warn("Not authorized to access checkpoints topic {} - this will degrade offset translation as fewer checkpoints may be emitted", config.checkpointsTopic(), error); + log.warn("Not authorized to access checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); } else { - log.info("Exception encountered loading Checkpoint topic {} - this will degrade offset translation as fewer checkpoints may be emitted", config.checkpointsTopic(), error); + log.info("Exception encountered loading checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); } } return checkpoints; @@ -165,9 +193,7 @@ void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback consumerGroups, - Map> idleConsumerGroupsOffset, - CheckpointsStore checkpointsStore) { + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, + Map> idleConsumerGroupsOffset, + CheckpointsStore checkpointsStore) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; @@ -110,8 +110,8 @@ public void start(Map props) { scheduler.execute(() -> { // loading the stores are potentially long running operations, so they run asynchronously // to avoid blocking task::start (until a task has completed starting it cannot be stopped) - checkpointsStore.start(); - offsetSyncStore.start(!checkpointsStore.loadSuccess()); + boolean checkpointsReadOk = checkpointsStore.start(); + offsetSyncStore.start(!checkpointsReadOk); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), @@ -178,8 +178,7 @@ private List sourceRecordsForGroup(String group) throws Interrupte long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); Map newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - Map oldCheckpoints = checkpointsStore.contents().computeIfAbsent(group, ignored -> new HashMap<>()); - oldCheckpoints.putAll(newCheckpoints); + checkpointsStore.update(group, newCheckpoints); return newCheckpoints.values().stream() .map(x -> checkpointRecord(x, timestamp)) .collect(Collectors.toList()); @@ -201,7 +200,7 @@ Map checkpointsForGroup(Map checkpoints = checkpointsStore.contents().get(checkpoint.consumerGroupId()); + Map checkpoints = checkpointsStore.get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; @@ -320,7 +319,7 @@ Map> syncGroupOffset() throws Exe Map> offsetToSyncAll = new HashMap<>(); // first, sync offsets for the idle consumers at target - for (Entry> group : getConvertedUpstreamOffset().entrySet()) { + for (Entry> group : checkpointsStore.computeConvertedUpstreamOffset().entrySet()) { String consumerGroupId = group.getKey(); // for each idle consumer at target, read the checkpoints (converted upstream offset) // from the pre-populated map @@ -397,18 +396,4 @@ void syncGroupOffset(String consumerGroupId, Map> getConvertedUpstreamOffset() { - Map> result = new HashMap<>(); - - for (Entry> entry : checkpointsStore.contents().entrySet()) { - String consumerId = entry.getKey(); - Map convertedUpstreamOffset = new HashMap<>(); - for (Checkpoint checkpoint : entry.getValue().values()) { - convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); - } - result.put(consumerId, convertedUpstreamOffset); - } - return result; - } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index a0dbde48fc68..69759a79a75c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -108,12 +108,16 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig */ public void start(boolean initializationMustReadToEnd) { this.initializationMustReadToEnd = initializationMustReadToEnd; - log.info("OffsetSyncStore initializationMustReadToEnd:{}{}", initializationMustReadToEnd, - initializationMustReadToEnd ? " - fewer checkpoints may be emitted" : ""); - backingStore.start(); + log.debug("OffsetSyncStore starting - must read to OffsetSync end = ", initializationMustReadToEnd); + backingStoreStart(); readToEnd = true; } + // overridable for testing + void backingStoreStart() { + backingStore.start(false); + } + OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { if (!readToEnd) { // If we have not read to the end of the syncs topic at least once, decline to translate any offsets. diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java index 7cd0c1aa2884..2c259e28b4a2 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java @@ -56,18 +56,15 @@ void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> expected = new HashMap<>(); expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); - assertEquals(expected, store.contents()); + assertEquals(expected, store.checkpointsPerConsumerGroup); } @Test @@ -86,15 +83,13 @@ void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); } - } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index d2f9029f9f92..75a649ba0802 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -41,7 +41,7 @@ public class MirrorCheckpointTaskTest { public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new CheckpointsStore(Collections.emptyMap())); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -60,10 +60,10 @@ public void testCheckpoint() { long t2UpstreamOffset = 7L; long t2DownstreamOffset = 8L; OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.readToEnd(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), + Collections.emptyMap(), new CheckpointsStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -164,7 +164,7 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, - new CheckpointsStore(checkpointsPerConsumerGroup)); + new CheckpointsStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -205,7 +205,7 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu @Test public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.readToEnd(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), new CheckpointsStore(Collections.emptyMap())); @@ -222,10 +222,10 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { @Test public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.readToEnd(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new CheckpointsStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -234,7 +234,7 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { @Test public void testCheckpointRecordsMonotonicIfStoreRewinds() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.readToEnd(); + offsetSyncStore.start(true); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), @@ -284,13 +284,17 @@ private Map assertCheckpointForTopic( public void testCheckpointsTaskRestartUsesExistingCheckpoints() { TopicPartition t1p0 = new TopicPartition("t1", 0); TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); - OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.setLoadingMustReadToEnd(false); - // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 - for (int i = 100; i <= 300; i += 100) { - offsetSyncStore.sync(t1p0, i, i); - } - offsetSyncStore.readToEnd(); + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 + for (int i = 100; i <= 300; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore.start(false); + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), new CheckpointsStore(Collections.emptyMap())); @@ -310,12 +314,15 @@ public void testCheckpointsTaskRestartUsesExistingCheckpoints() { // Simulate task restart, during which more offsets are added to the sync topic, and thus the // corresponding OffsetSyncStore no longer has a mapping for 100->100 // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 - offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.setLoadingMustReadToEnd(false); - for (int i = 175; i <= 475; i += 100) { - offsetSyncStore.sync(t1p0, i, i); - } - offsetSyncStore.readToEnd(); + offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int i = 175; i <= 475; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore.start(false); // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) Map> checkpointsPerConsumerGroup = new HashMap<>(); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index a3eeee270f5b..217aecc2a64b 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -39,12 +39,9 @@ static class FakeOffsetSyncStore extends OffsetSyncStore { super(); } - public void setLoadingMustReadToEnd(boolean b) { - this.initializationMustReadToEnd = false; - } - - public void readToEnd() { - this.readToEnd = true; + @Override + void backingStoreStart() { + // do not start KafkaBasedLog } void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOffset) { @@ -59,7 +56,7 @@ void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOff @Test public void testOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.readToEnd(); + store.start(true); // Emit synced downstream offset without dead-reckoning store.sync(tp, 100, 200); @@ -97,7 +94,7 @@ public void testNoTranslationIfStoreNotStarted() { assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); // After the store is started all offsets are visible - store.readToEnd(); + store.start(true); assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); @@ -107,7 +104,7 @@ public void testNoTranslationIfStoreNotStarted() { @Test public void testNoTranslationIfNoOffsetSync() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.readToEnd(); + store.start(true); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); } } @@ -121,7 +118,7 @@ public void testPastOffsetTranslation() { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); } - store.readToEnd(); + store.start(true); // After starting but before seeing new offsets, only the latest startup offset can be translated assertSparseSync(store, 1000, -1); @@ -160,50 +157,53 @@ public void testPastOffsetTranslation() { // this test has been wriiten knowing the exact offsets syncs stored @Test public void testPastOffsetTranslationWithoutInitializationReadToEnd() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - int maxOffsetLag = 10; - int offset = 0; - store.setLoadingMustReadToEnd(false); - for (; offset <= 1000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); - } - store.readToEnd(); - - // After starting but before seeing new offsets - assertTranslationsNearby(store, 400, 480, 0); - assertTranslationsNearby(store, 500, 720, 480); - assertTranslationsNearby(store, 1000, 1000, 990); - - for (; offset <= 10000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + 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); + } } + }; - // 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); + store.start(false); - // 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); + // After starting but before seeing new offsets + assertTranslationsNearby(store, 400, 480, 0); + assertTranslationsNearby(store, 500, 720, 480); + assertTranslationsNearby(store, 1000, 1000, 990); - // Rewinding upstream offsets should clear all historical syncs - store.sync(tp, 1500, 11000); + for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); - assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); - assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); - assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } + + // 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); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); + assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } @Test @@ -266,7 +266,7 @@ public void testDroppedSyncsSpacing() { */ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream steps, int maximumExpirations) { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.readToEnd(); + store.start(true); store.sync(tp, firstOffset, firstOffset); PrimitiveIterator.OfLong iterator = steps.iterator(); long offset = firstOffset; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index cb44ca846b9d..c1f19e334910 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -108,7 +108,6 @@ public class KafkaBasedLog { private final Queue> readLogEndOffsetCallbacks; private final java.util.function.Consumer initializer; // initialized as false for backward compatibility - // TODO make the reportErrorsToCallback=true a default in Kafka 4.0 ? private volatile boolean reportErrorsToCallback = false; /** @@ -248,6 +247,7 @@ public void stop() { public void start() { start(false); } + public void start(boolean reportErrorsToCallback) { this.reportErrorsToCallback = reportErrorsToCallback; log.info("Starting KafkaBasedLog with topic {} reportErrorsToCallback={}", topic, reportErrorsToCallback); From e415e674ca604d0a7e6cd90403eed5e9ea91de52 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Fri, 17 May 2024 13:08:07 +0100 Subject: [PATCH 08/15] one more unit test --- .../connect/mirror/MirrorCheckpointTask.java | 6 ++-- .../mirror/MirrorCheckpointTaskTest.java | 30 +++++++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index e8b12dcd1158..74d073a3b2d1 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -86,6 +86,8 @@ public MirrorCheckpointTask() {} this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; this.checkpointsStore = checkpointsStore; this.topicFilter = topic -> true; + this.interval = Duration.ofNanos(1); + this.pollTimeout = Duration.ofNanos(1); } @Override @@ -172,8 +174,8 @@ public List poll() throws InterruptedException { } } - - private List sourceRecordsForGroup(String group) throws InterruptedException { + // visible for testing + List sourceRecordsForGroup(String group) throws InterruptedException { try { long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index 75a649ba0802..6c512a955c9e 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Collections; import java.util.Optional; @@ -33,8 +34,12 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + public class MirrorCheckpointTaskTest { @Test @@ -352,4 +357,29 @@ void backingStoreStart() { assertEquals(400, cp.upstreamOffset()); assertEquals(376, cp.downstreamOffset()); } + + @Test + public void testCheckpointStoreInitialized() throws InterruptedException { + CheckpointsStore checkpointsStore = mock(CheckpointsStore.class); + + MirrorCheckpointTask task = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), + new OffsetSyncStoreTest.FakeOffsetSyncStore(), + Collections.singleton("group"), + Collections.emptyMap(), + checkpointsStore) { + + @Override + List sourceRecordsForGroup(String group) throws InterruptedException { + SourceRecord sr = new SourceRecord(Collections.emptyMap(), Collections.emptyMap(), "", 0, null, null); + return Collections.singletonList(sr); + } + }; + + assertNull(task.poll()); + + when(checkpointsStore.isInitialized()).thenReturn(true); + List polled = task.poll(); + assertEquals(1, polled.size()); + } } From 56db4edf332d105e8ed580dee67fe0b32fd3fd75 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Mon, 20 May 2024 17:23:26 +0100 Subject: [PATCH 09/15] Renamed CheckpointsStore to CheckpointStore for consistency with OffsetSyncStore CheckpointStore started debug/trace message --- ...kpointsStore.java => CheckpointStore.java} | 14 +++++++---- .../connect/mirror/MirrorCheckpointTask.java | 20 ++++++++-------- ...toreTest.java => CheckpointStoreTest.java} | 6 ++--- .../mirror/MirrorCheckpointTaskTest.java | 24 +++++++++---------- 4 files changed, 34 insertions(+), 30 deletions(-) rename connect/mirror/src/main/java/org/apache/kafka/connect/mirror/{CheckpointsStore.java => CheckpointStore.java} (93%) rename connect/mirror/src/test/java/org/apache/kafka/connect/mirror/{CheckpointsStoreTest.java => CheckpointStoreTest.java} (95%) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java similarity index 93% rename from connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java rename to connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java index 865135e26bc6..24209b1327df 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointsStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java @@ -44,9 +44,9 @@ * The Kafka log is closed after the initial load and only the in memory map is * used after start. */ -class CheckpointsStore implements AutoCloseable { +class CheckpointStore implements AutoCloseable { - private static final Logger log = LoggerFactory.getLogger(CheckpointsStore.class); + private static final Logger log = LoggerFactory.getLogger(CheckpointStore.class); private final MirrorCheckpointTaskConfig config; private final Set consumerGroups; @@ -59,13 +59,13 @@ class CheckpointsStore implements AutoCloseable { private volatile boolean loadSuccess = false; private volatile boolean isInitialized = false; - CheckpointsStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + CheckpointStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { this.config = config; this.consumerGroups = new HashSet<>(consumerGroups); } // constructor for testing only - CheckpointsStore(Map> checkpointsPerConsumerGroup) { + CheckpointStore(Map> checkpointsPerConsumerGroup) { this.config = null; //ignored by tests this.consumerGroups = null; //ignored by tests this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; @@ -77,7 +77,11 @@ class CheckpointsStore implements AutoCloseable { boolean start() { checkpointsPerConsumerGroup = readCheckpoints(); isInitialized = true; - log.trace("CheckpointsStore started, load success={}, map={}", loadSuccess, checkpointsPerConsumerGroup); + if (log.isTraceEnabled()) { + log.trace("CheckpointStore started, load success={}, map={}", loadSuccess, checkpointsPerConsumerGroup); + } else { + log.debug("CheckpointStore started, load success={}, map.size={}", loadSuccess, checkpointsPerConsumerGroup.size()); + } return loadSuccess; } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 74d073a3b2d1..7f446efea5df 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -69,7 +69,7 @@ public class MirrorCheckpointTask extends SourceTask { private MirrorCheckpointMetrics metrics; private Scheduler scheduler; private Map> idleConsumerGroupsOffset; - private CheckpointsStore checkpointsStore; + private CheckpointStore checkpointStore; public MirrorCheckpointTask() {} @@ -77,14 +77,14 @@ public MirrorCheckpointTask() {} MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, Map> idleConsumerGroupsOffset, - CheckpointsStore checkpointsStore) { + CheckpointStore checkpointStore) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; this.offsetSyncStore = offsetSyncStore; this.consumerGroups = consumerGroups; this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; - this.checkpointsStore = checkpointsStore; + this.checkpointStore = checkpointStore; this.topicFilter = topic -> true; this.interval = Duration.ofNanos(1); this.pollTimeout = Duration.ofNanos(1); @@ -107,12 +107,12 @@ public void start(Map props) { targetAdminClient = config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin")); metrics = config.metrics(); idleConsumerGroupsOffset = new HashMap<>(); - checkpointsStore = new CheckpointsStore(config, consumerGroups); + checkpointStore = new CheckpointStore(config, consumerGroups); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { // loading the stores are potentially long running operations, so they run asynchronously // to avoid blocking task::start (until a task has completed starting it cannot be stopped) - boolean checkpointsReadOk = checkpointsStore.start(); + boolean checkpointsReadOk = checkpointStore.start(); offsetSyncStore.start(!checkpointsReadOk); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); @@ -133,7 +133,7 @@ public void stop() { long start = System.currentTimeMillis(); stopping = true; Utils.closeQuietly(topicFilter, "topic filter"); - Utils.closeQuietly(checkpointsStore, "checkpoints store"); + Utils.closeQuietly(checkpointStore, "checkpoints store"); Utils.closeQuietly(offsetSyncStore, "offset sync store"); Utils.closeQuietly(sourceAdminClient, "source admin client"); Utils.closeQuietly(targetAdminClient, "target admin client"); @@ -154,7 +154,7 @@ public List poll() throws InterruptedException { while (!stopping && System.currentTimeMillis() < deadline) { Thread.sleep(pollTimeout.toMillis()); } - if (stopping || !checkpointsStore.isInitialized()) { + if (stopping || !checkpointStore.isInitialized()) { // we are stopping, or not fully initialized, return early. return null; } @@ -180,7 +180,7 @@ List sourceRecordsForGroup(String group) throws InterruptedExcepti long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); Map newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - checkpointsStore.update(group, newCheckpoints); + checkpointStore.update(group, newCheckpoints); return newCheckpoints.values().stream() .map(x -> checkpointRecord(x, timestamp)) .collect(Collectors.toList()); @@ -202,7 +202,7 @@ Map checkpointsForGroup(Map checkpoints = checkpointsStore.get(checkpoint.consumerGroupId()); + Map checkpoints = checkpointStore.get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; @@ -321,7 +321,7 @@ Map> syncGroupOffset() throws Exe Map> offsetToSyncAll = new HashMap<>(); // first, sync offsets for the idle consumers at target - for (Entry> group : checkpointsStore.computeConvertedUpstreamOffset().entrySet()) { + for (Entry> group : checkpointStore.computeConvertedUpstreamOffset().entrySet()) { String consumerGroupId = group.getKey(); // for each idle consumer at target, read the checkpoints (converted upstream offset) // from the pre-populated map diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java similarity index 95% rename from connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java rename to connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java index 2c259e28b4a2..5255bbb355ad 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointsStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java @@ -34,7 +34,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class CheckpointsStoreTest { +public class CheckpointStoreTest { @Test public void testReadCheckpointsTopic() { @@ -44,7 +44,7 @@ public void testReadCheckpointsTopic() { MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); - CheckpointsStore store = new CheckpointsStore(config, consumerGroups) { + CheckpointStore store = new CheckpointStore(config, consumerGroups) { @Override void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); @@ -75,7 +75,7 @@ public void testReadCheckpointsTopicError() { MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); - CheckpointsStore store = new CheckpointsStore(config, consumerGroups) { + CheckpointStore store = new CheckpointStore(config, consumerGroups) { @Override void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index 6c512a955c9e..2ae9af38174b 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -46,7 +46,7 @@ public class MirrorCheckpointTaskTest { public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new CheckpointStore(Collections.emptyMap())); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -68,7 +68,7 @@ public void testCheckpoint() { offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), - Collections.emptyMap(), new CheckpointsStore(Collections.emptyMap())); + Collections.emptyMap(), new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -169,7 +169,7 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, - new CheckpointsStore(checkpointsPerConsumerGroup)); + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -200,7 +200,7 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source", "target", new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, - new CheckpointsStore(checkpointsPerConsumerGroup)); + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -213,7 +213,7 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); Optional checkpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 1), @@ -230,7 +230,7 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -243,7 +243,7 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(checkpointsPerConsumerGroup)); + new CheckpointStore(checkpointsPerConsumerGroup)); TopicPartition tp = new TopicPartition("topic1", 0); TopicPartition targetTP = new TopicPartition("source1.topic1", 0); @@ -302,7 +302,7 @@ void backingStoreStart() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(Collections.emptyMap())); + new CheckpointStore(Collections.emptyMap())); // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 // (as nearest mapping in OffsetSyncStore is 200->200) @@ -334,7 +334,7 @@ void backingStoreStart() { checkpointsPerConsumerGroup.put("group1", result); mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), - new CheckpointsStore(checkpointsPerConsumerGroup)); + new CheckpointStore(checkpointsPerConsumerGroup)); // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. @@ -360,14 +360,14 @@ void backingStoreStart() { @Test public void testCheckpointStoreInitialized() throws InterruptedException { - CheckpointsStore checkpointsStore = mock(CheckpointsStore.class); + CheckpointStore checkpointStore = mock(CheckpointStore.class); MirrorCheckpointTask task = new MirrorCheckpointTask("source1", "target2", new DefaultReplicationPolicy(), new OffsetSyncStoreTest.FakeOffsetSyncStore(), Collections.singleton("group"), Collections.emptyMap(), - checkpointsStore) { + checkpointStore) { @Override List sourceRecordsForGroup(String group) throws InterruptedException { @@ -378,7 +378,7 @@ List sourceRecordsForGroup(String group) throws InterruptedExcepti assertNull(task.poll()); - when(checkpointsStore.isInitialized()).thenReturn(true); + when(checkpointStore.isInitialized()).thenReturn(true); List polled = task.poll(); assertEquals(1, polled.size()); } From 36b7bb1ab39c0f0e4bf84a0208d007f085d6a5d1 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Tue, 21 May 2024 16:14:53 +0100 Subject: [PATCH 10/15] Addressed review comments --- .../kafka/connect/mirror/CheckpointStore.java | 16 +- .../kafka/connect/mirror/OffsetSyncStore.java | 6 +- .../mirror/MirrorCheckpointTaskTest.java | 40 ++--- .../connect/mirror/OffsetSyncStoreTest.java | 170 +++++++++--------- 4 files changed, 106 insertions(+), 126 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java index 24209b1327df..cbe76efecb38 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java @@ -44,7 +44,7 @@ * The Kafka log is closed after the initial load and only the in memory map is * used after start. */ -class CheckpointStore implements AutoCloseable { +public class CheckpointStore implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(CheckpointStore.class); @@ -59,7 +59,7 @@ class CheckpointStore implements AutoCloseable { private volatile boolean loadSuccess = false; private volatile boolean isInitialized = false; - CheckpointStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + public CheckpointStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { this.config = config; this.consumerGroups = new HashSet<>(consumerGroups); } @@ -74,7 +74,7 @@ class CheckpointStore implements AutoCloseable { } // potentially long running - boolean start() { + public boolean start() { checkpointsPerConsumerGroup = readCheckpoints(); isInitialized = true; if (log.isTraceEnabled()) { @@ -85,21 +85,21 @@ boolean start() { return loadSuccess; } - boolean isInitialized() { + public boolean isInitialized() { return isInitialized; } - void update(String group, Map newCheckpoints) { + public void update(String group, Map newCheckpoints) { Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); oldCheckpoints.putAll(newCheckpoints); } - Map get(String group) { + public Map get(String group) { Map result = checkpointsPerConsumerGroup.get(group); return result == null ? null : Collections.unmodifiableMap(result); } - Map> computeConvertedUpstreamOffset() { + public Map> computeConvertedUpstreamOffset() { Map> result = new HashMap<>(); for (Map.Entry> entry : checkpointsPerConsumerGroup.entrySet()) { @@ -129,7 +129,7 @@ private void releaseResources() { // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog // e.g. unauthorized to read from topic (non-retriable) // if any are encountered, treat the loading of Checkpoints as failed. - Map> readCheckpoints() { + private Map> readCheckpoints() { Map> checkpoints = new HashMap<>(); Callback> consumedCallback = (error, cpRecord) -> { if (error != null) { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index 69759a79a75c..1716d201e2ee 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -54,7 +54,7 @@ * started after the position of the consumer group, or if relevant offset syncs for the topic were potentially used as * for translation in an earlier generation of the sync store. */ -class OffsetSyncStore implements AutoCloseable { +public class OffsetSyncStore implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(OffsetSyncStore.class); // Store one offset sync for each bit of the topic offset. @@ -66,7 +66,7 @@ class OffsetSyncStore implements AutoCloseable { protected volatile boolean initializationMustReadToEnd = true; protected volatile boolean readToEnd = false; - OffsetSyncStore(MirrorCheckpointConfig config) { + public OffsetSyncStore(MirrorCheckpointConfig config) { Consumer consumer = null; TopicAdmin admin = null; KafkaBasedLog store; @@ -118,7 +118,7 @@ void backingStoreStart() { backingStore.start(false); } - OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { + public OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { if (!readToEnd) { // If we have not read to the end of the syncs topic at least once, decline to translate any offsets. // This prevents emitting stale offsets while initially reading the offset syncs topic. diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index 2ae9af38174b..caba7ba5bc00 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -33,7 +33,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -308,18 +307,14 @@ void backingStoreStart() { // (as nearest mapping in OffsetSyncStore is 200->200) Map upstreamGroupOffsets = new HashMap<>(); upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); - Map result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); - - assertEquals(1, result.size()); - Checkpoint cp = result.get(sourceT1p0); - assertNotNull(cp); - assertEquals(250, cp.upstreamOffset()); - assertEquals(201, cp.downstreamOffset()); + Map checkpoints = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 250, 201, ""), checkpoints.get(sourceT1p0)); // Simulate task restart, during which more offsets are added to the sync topic, and thus the // corresponding OffsetSyncStore no longer has a mapping for 100->100 // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 - offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore2 = new OffsetSyncStoreTest.FakeOffsetSyncStore() { @Override void backingStoreStart() { for (int i = 175; i <= 475; i += 100) { @@ -327,35 +322,30 @@ void backingStoreStart() { } } }; - offsetSyncStore.start(false); + offsetSyncStore2.start(false); // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) Map> checkpointsPerConsumerGroup = new HashMap<>(); - checkpointsPerConsumerGroup.put("group1", result); - mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + checkpointsPerConsumerGroup.put("group1", checkpoints); + MirrorCheckpointTask mirrorCheckpointTask2 = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore2, Collections.emptySet(), Collections.emptyMap(), new CheckpointStore(checkpointsPerConsumerGroup)); // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. - assertEquals(OptionalLong.of(176), offsetSyncStore.translateDownstream(null, t1p0, 250)); - assertEquals(OptionalLong.of(176), offsetSyncStore.translateDownstream(null, t1p0, 370)); + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 250)); + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 370)); upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); - result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); - assertEquals(0, result.size()); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(370)); - result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); - assertEquals(0, result.size()); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); // Upstream offset 400 has a closes downstream value of 376, and is emitted because it has // a later downstream offset than the last checkpoint's downstream (201) upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(400)); - result = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); - assertEquals(1, result.size()); - cp = result.get(sourceT1p0); - assertNotNull(cp); - assertEquals(400, cp.upstreamOffset()); - assertEquals(376, cp.downstreamOffset()); + Map checkpoints2 = mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints2.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 400, 376, ""), checkpoints2.get(sourceT1p0)); } @Test diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index 217aecc2a64b..4983622a8f28 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -34,9 +34,12 @@ public class OffsetSyncStoreTest { static TopicPartition tp = new TopicPartition("topic1", 2); static class FakeOffsetSyncStore extends OffsetSyncStore { + private boolean startCalled = false; - FakeOffsetSyncStore() { - super(); + @Override + public void start(boolean initializationMustReadToEnd) { + startCalled = true; + super.start(initializationMustReadToEnd); } @Override @@ -44,7 +47,9 @@ void backingStoreStart() { // do not start KafkaBasedLog } + // simulate OffsetSync load as from KafkaBasedLog void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOffset) { + assertTrue(startCalled); // sync in tests should only be called after store.start OffsetSync offsetSync = new OffsetSync(topicPartition, upstreamOffset, downstreamOffset); byte[] key = offsetSync.recordKey(); byte[] value = offsetSync.recordValue(); @@ -81,24 +86,28 @@ public void testOffsetTranslation() { @Test public void testNoTranslationIfStoreNotStarted() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - // no offsets exist and store is not started - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); + FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // read a sync during startup + sync(tp, 100, 200); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 200)); + } + }; - // read a sync during startup - store.sync(tp, 100, 200); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); + // no offsets exist and store is not started + assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // After the store is started all offsets are visible - store.start(true); - assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); - } + // After the store is started all offsets are visible + store.start(true); + + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); } @Test @@ -111,50 +120,52 @@ public void testNoTranslationIfNoOffsetSync() { @Test public void testPastOffsetTranslation() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - int maxOffsetLag = 10; - int offset = 0; - for (; offset <= 1000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + 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(true); - - // After starting but before seeing new offsets, only the latest startup offset can be translated - assertSparseSync(store, 1000, -1); + }; + store.start(true); - for (; offset <= 10000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); - } + // After starting but before seeing new offsets, only the latest startup offset can be translated + assertSparseSync(store, 1000, -1); - // After seeing new offsets, we still cannot translate earlier than the latest startup offset - // Invariant D: the last sync from the initial read-to-end is still stored - assertSparseSync(store, 1000, -1); - - // 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 - assertSparseSync(store, 4840, 1000); - assertSparseSync(store, 6760, 4840); - assertSparseSync(store, 8680, 6760); - assertSparseSync(store, 9160, 8680); - assertSparseSync(store, 9640, 9160); - assertSparseSync(store, 9880, 9640); - assertSparseSync(store, 9940, 9880); - assertSparseSync(store, 9970, 9940); - assertSparseSync(store, 9990, 9970); - assertSparseSync(store, 10000, 9990); - - // Rewinding upstream offsets should clear all historical syncs - store.sync(tp, 1500, 11000); + for (int offset = 1000 + maxOffsetLag; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); - assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); - assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); - assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } + + // After seeing new offsets, we still cannot translate earlier than the latest startup offset + // Invariant D: the last sync from the initial read-to-end is still stored + assertSparseSync(store, 1000, -1); + + // 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 + assertSparseSync(store, 4840, 1000); + assertSparseSync(store, 6760, 4840); + assertSparseSync(store, 8680, 6760); + assertSparseSync(store, 9160, 8680); + assertSparseSync(store, 9640, 9160); + assertSparseSync(store, 9880, 9640); + assertSparseSync(store, 9940, 9880); + assertSparseSync(store, 9970, 9940); + assertSparseSync(store, 9990, 9970); + assertSparseSync(store, 10000, 9990); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); + assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } - // this test has been wriiten knowing the exact offsets syncs stored + // this test has been written knowing the exact offsets syncs stored @Test public void testPastOffsetTranslationWithoutInitializationReadToEnd() { final int maxOffsetLag = 10; @@ -172,9 +183,9 @@ void backingStoreStart() { 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); + 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); @@ -182,21 +193,17 @@ void backingStoreStart() { } // 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); + // 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); // Rewinding upstream offsets should clear all historical syncs store.sync(tp, 1500, 11000); @@ -297,23 +304,6 @@ private void assertSparseSync(FakeOffsetSyncStore store, long syncOffset, long p assertEquals(OptionalLong.of(syncOffset + 1), store.translateDownstream(null, tp, syncOffset + 2)); } - private void assertTranslationsNearby(FakeOffsetSyncStore store, long syncOffsetToTranslate, long nearestLargeOrEqualsStoredOffset, long previousStoredOffset) { - // sanity test - assertTrue(syncOffsetToTranslate > previousStoredOffset); - - if (syncOffsetToTranslate == nearestLargeOrEqualsStoredOffset) { - assertEquals(OptionalLong.of(previousStoredOffset == -1 ? previousStoredOffset : previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate - 1)); - assertEquals(OptionalLong.of(syncOffsetToTranslate), store.translateDownstream(null, tp, syncOffsetToTranslate)); - assertEquals(OptionalLong.of(syncOffsetToTranslate + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 1)); - assertEquals(OptionalLong.of(syncOffsetToTranslate + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 2)); - } else if (syncOffsetToTranslate < nearestLargeOrEqualsStoredOffset) { - assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate - 1)); - assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate)); - assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 1)); - assertEquals(OptionalLong.of(previousStoredOffset + 1), store.translateDownstream(null, tp, syncOffsetToTranslate + 2)); - } - } - private int countDistinctStoredSyncs(FakeOffsetSyncStore store, TopicPartition topicPartition) { int count = 1; for (int i = 1; i < OffsetSyncStore.SYNCS_PER_PARTITION; i++) { From dfeca994360025ec8878c1afd064e9c36ff3d868 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Tue, 21 May 2024 23:07:59 +0100 Subject: [PATCH 11/15] avoid warning --- .../connect/mirror/OffsetSyncStoreTest.java | 146 +++++++++--------- 1 file changed, 74 insertions(+), 72 deletions(-) diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index 4983622a8f28..3f2ddbc62e93 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -86,7 +86,7 @@ public void testOffsetTranslation() { @Test public void testNoTranslationIfStoreNotStarted() { - FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { @Override void backingStoreStart() { // read a sync during startup @@ -95,19 +95,19 @@ void backingStoreStart() { assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 100)); assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 200)); } - }; - - // no offsets exist and store is not started - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); + }) { + // no offsets exist and store is not started + assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // After the store is started all offsets are visible - store.start(true); + // After the store is started all offsets are visible + store.start(true); - assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); + } } @Test @@ -121,7 +121,7 @@ public void testNoTranslationIfNoOffsetSync() { @Test public void testPastOffsetTranslation() { int maxOffsetLag = 10; - FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { @Override void backingStoreStart() { for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { @@ -129,40 +129,41 @@ void backingStoreStart() { assertSparseSyncInvariant(this, tp); } } - }; - store.start(true); + }) { + store.start(true); - // After starting but before seeing new offsets, only the latest startup offset can be translated - assertSparseSync(store, 1000, -1); + // After starting but before seeing new offsets, only the latest startup offset can be translated + assertSparseSync(store, 1000, -1); - for (int offset = 1000 + maxOffsetLag; offset <= 10000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); + for (int offset = 1000 + maxOffsetLag; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); + } + + // After seeing new offsets, we still cannot translate earlier than the latest startup offset + // Invariant D: the last sync from the initial read-to-end is still stored + assertSparseSync(store, 1000, -1); + + // 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 + assertSparseSync(store, 4840, 1000); + assertSparseSync(store, 6760, 4840); + assertSparseSync(store, 8680, 6760); + assertSparseSync(store, 9160, 8680); + assertSparseSync(store, 9640, 9160); + assertSparseSync(store, 9880, 9640); + assertSparseSync(store, 9940, 9880); + assertSparseSync(store, 9970, 9940); + assertSparseSync(store, 9990, 9970); + assertSparseSync(store, 10000, 9990); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } - - // After seeing new offsets, we still cannot translate earlier than the latest startup offset - // Invariant D: the last sync from the initial read-to-end is still stored - assertSparseSync(store, 1000, -1); - - // 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 - assertSparseSync(store, 4840, 1000); - assertSparseSync(store, 6760, 4840); - assertSparseSync(store, 8680, 6760); - assertSparseSync(store, 9160, 8680); - assertSparseSync(store, 9640, 9160); - assertSparseSync(store, 9880, 9640); - assertSparseSync(store, 9940, 9880); - assertSparseSync(store, 9970, 9940); - assertSparseSync(store, 9990, 9970); - assertSparseSync(store, 10000, 9990); - - // Rewinding upstream offsets should clear all historical syncs - store.sync(tp, 1500, 11000); - assertSparseSyncInvariant(store, tp); - assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); - assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); - assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } // this test has been written knowing the exact offsets syncs stored @@ -170,7 +171,7 @@ void backingStoreStart() { public void testPastOffsetTranslationWithoutInitializationReadToEnd() { final int maxOffsetLag = 10; - FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { @Override void backingStoreStart() { for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { @@ -178,39 +179,40 @@ void backingStoreStart() { assertSparseSyncInvariant(this, tp); } } - }; + }) { + + store.start(false); - store.start(false); + // After starting but before seeing new offsets + assertSparseSync(store, 480, 0); + assertSparseSync(store, 720, 480); + assertSparseSync(store, 1000, 990); - // 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); + } - for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); + // 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); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } - - // 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); - - // Rewinding upstream offsets should clear all historical syncs - store.sync(tp, 1500, 11000); - assertSparseSyncInvariant(store, tp); - assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); - assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); - assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); } @Test From 2fdc9f9defa84dc028ea3b5552a2233dd5e78c5a Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Tue, 21 May 2024 23:15:30 +0100 Subject: [PATCH 12/15] avoid warning --- .../apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index caba7ba5bc00..0afc4f74f2f9 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -264,7 +264,7 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); - offsetSyncStore.sync(tp, upstream++, downstream++); + offsetSyncStore.sync(tp, upstream, downstream); // The OffsetSyncStore will change its translation of the same offset assertNotEquals(OptionalLong.of(expectedDownstreamOffset), offsetSyncStore.translateDownstream("g1", tp, consumerGroupOffset)); // But the task will filter this out and not emit a checkpoint @@ -360,7 +360,7 @@ public void testCheckpointStoreInitialized() throws InterruptedException { checkpointStore) { @Override - List sourceRecordsForGroup(String group) throws InterruptedException { + List sourceRecordsForGroup(String group) { SourceRecord sr = new SourceRecord(Collections.emptyMap(), Collections.emptyMap(), "", 0, null, null); return Collections.singletonList(sr); } From 98966536a22a6b32158472d2bab4b9d69384cb30 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Tue, 21 May 2024 23:23:16 +0100 Subject: [PATCH 13/15] avoid warning --- .../connect/mirror/CheckpointStoreTest.java | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java index 5255bbb355ad..b7b3904899f1 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java @@ -44,7 +44,7 @@ public void testReadCheckpointsTopic() { MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); - CheckpointStore store = new CheckpointStore(config, consumerGroups) { + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { @Override void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); @@ -55,16 +55,17 @@ void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); } - }; - assertFalse(store.isInitialized()); + }) { + assertFalse(store.isInitialized()); - assertTrue(store.start(), "expected start to return success"); - assertTrue(store.isInitialized()); + assertTrue(store.start(), "expected start to return success"); + assertTrue(store.isInitialized()); - Map> expected = new HashMap<>(); - expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), - new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); - assertEquals(expected, store.checkpointsPerConsumerGroup); + Map> expected = new HashMap<>(); + expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), + new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); + assertEquals(expected, store.checkpointsPerConsumerGroup); + } } @Test @@ -75,17 +76,18 @@ public void testReadCheckpointsTopicError() { MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); - CheckpointStore store = new CheckpointStore(config, consumerGroups) { + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { @Override void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { - consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "topic", 1, 0, 0)); consumedCallback.onCompletion(new TopicAuthorizationException("test"), null); } - }; + }) { - assertFalse(store.start(), "expected start to return failure"); - assertTrue(store.isInitialized()); - assertTrue(store.checkpointsPerConsumerGroup.isEmpty()); + assertFalse(store.start(), "expected start to return failure"); + assertTrue(store.isInitialized()); + assertTrue(store.checkpointsPerConsumerGroup.isEmpty()); + } } ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { From 1d0363bda90210f178a6470f031fdc3b801c4908 Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Wed, 22 May 2024 09:58:21 +0100 Subject: [PATCH 14/15] debug log message oops --- .../java/org/apache/kafka/connect/mirror/OffsetSyncStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index 1716d201e2ee..c4d91d8ea207 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -108,7 +108,7 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig */ public void start(boolean initializationMustReadToEnd) { this.initializationMustReadToEnd = initializationMustReadToEnd; - log.debug("OffsetSyncStore starting - must read to OffsetSync end = ", initializationMustReadToEnd); + log.debug("OffsetSyncStore starting - must read to OffsetSync end = {}", initializationMustReadToEnd); backingStoreStart(); readToEnd = true; } From b6eeaba62e178cc83771dbfa96f7aa22ba497a6f Mon Sep 17 00:00:00 2001 From: Edoardo Comar Date: Wed, 22 May 2024 17:32:39 +0100 Subject: [PATCH 15/15] avoid java21 this-escape warning --- .../org/apache/kafka/connect/mirror/OffsetSyncStore.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index c4d91d8ea207..16038044ddd2 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -66,7 +66,8 @@ public class OffsetSyncStore implements AutoCloseable { protected volatile boolean initializationMustReadToEnd = true; protected volatile boolean readToEnd = false; - public OffsetSyncStore(MirrorCheckpointConfig config) { + // package access to avoid Java 21 "this-escape" warning + OffsetSyncStore(MirrorCheckpointConfig config) { Consumer consumer = null; TopicAdmin admin = null; KafkaBasedLog store; @@ -98,6 +99,7 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig ); } + // for testing OffsetSyncStore() { this.admin = null; this.backingStore = null; @@ -219,7 +221,6 @@ private void clearSyncArray(OffsetSync[] syncs, OffsetSync offsetSync) { private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSync offsetSync) { long upstreamOffset = offsetSync.upstreamOffset(); - // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all.