gharris1727 commented on code in PR #15910: URL: https://github.com/apache/kafka/pull/15910#discussion_r1603861223
########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java: ########## @@ -395,7 +401,7 @@ void syncGroupOffset(String consumerGroupId, Map<TopicPartition, OffsetAndMetada Map<String, Map<TopicPartition, OffsetAndMetadata>> getConvertedUpstreamOffset() { Map<String, Map<TopicPartition, OffsetAndMetadata>> result = new HashMap<>(); - for (Entry<String, Map<TopicPartition, Checkpoint>> entry : checkpointsPerConsumerGroup.entrySet()) { + for (Entry<String, Map<TopicPartition, Checkpoint>> entry : checkpointsStore.contents().entrySet()) { String consumerId = entry.getKey(); Review Comment: this getConvertedUpstreamOffset could be moved to CheckpointStore since it only depends on the checkpoint store state. ########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java: ########## @@ -69,20 +69,22 @@ public class MirrorCheckpointTask extends SourceTask { private MirrorCheckpointMetrics metrics; private Scheduler scheduler; private Map<String, Map<TopicPartition, OffsetAndMetadata>> idleConsumerGroupsOffset; - private Map<String, Map<TopicPartition, Checkpoint>> checkpointsPerConsumerGroup; + private CheckpointsStore checkpointsStore; + public MirrorCheckpointTask() {} // for testing MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, - ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, - Map<String, Map<TopicPartition, OffsetAndMetadata>> idleConsumerGroupsOffset, - Map<String, Map<TopicPartition, Checkpoint>> checkpointsPerConsumerGroup) { + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set<String> consumerGroups, + Map<String, Map<TopicPartition, OffsetAndMetadata>> idleConsumerGroupsOffset, + CheckpointsStore checkpointsStore) { Review Comment: nit: indenting ########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java: ########## @@ -105,7 +106,10 @@ private KafkaBasedLog<byte[], byte[]> createBackingStore(MirrorCheckpointConfig /** * Start the OffsetSyncStore, blocking until all previous Offset Syncs have been read from backing storage. */ - public void start() { + public void start(boolean initializationMustReadToEnd) { + this.initializationMustReadToEnd = initializationMustReadToEnd; + log.info("OffsetSyncStore initializationMustReadToEnd:{}{}", initializationMustReadToEnd, + initializationMustReadToEnd ? " - fewer checkpoints may be emitted" : ""); Review Comment: nit: Make this more verbose and user-oriented. They don't care that the variable is called initializationMustReadToEnd, and "Must read to end" is a very technical description of what is happening here. Specify more precisely which checkpoints aren't being emitted. Fewer could mean every other one, but it's actually offsets which were mirrored before the task started. Actually, this message works better if you put it after the backingStore.start() call: You can print out the oldest offset sync to say that translation is starting there, and whether this is limited by the initialization setting. ########## 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() { Review Comment: bump on this comment, now that we're converging on the final design. ########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java: ########## @@ -172,7 +178,7 @@ private List<SourceRecord> sourceRecordsForGroup(String group) throws Interrupte long timestamp = System.currentTimeMillis(); Map<TopicPartition, OffsetAndMetadata> upstreamGroupOffsets = listConsumerGroupOffsets(group); Map<TopicPartition, Checkpoint> newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - Map<TopicPartition, Checkpoint> oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); + Map<TopicPartition, Checkpoint> oldCheckpoints = checkpointsStore.contents().computeIfAbsent(group, ignored -> new HashMap<>()); oldCheckpoints.putAll(newCheckpoints); Review Comment: Move this to a new `CheckpointsStore#emitCheckpoints(Map<TopicPartition, Checkpoint>)` method ########## connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java: ########## @@ -195,7 +201,7 @@ Map<TopicPartition, Checkpoint> checkpointsForGroup(Map<TopicPartition, OffsetAn } private boolean checkpointIsMoreRecent(Checkpoint checkpoint) { - Map<TopicPartition, Checkpoint> checkpoints = checkpointsPerConsumerGroup.get(checkpoint.consumerGroupId()); + Map<TopicPartition, Checkpoint> checkpoints = checkpointsStore.contents().get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; Review Comment: This can be moved to a new `CheckpointStore#get(String, TopicPartition)` method. ########## 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<String> consumerGroups; + + private TopicAdmin cpAdmin = null; + private KafkaBasedLog<byte[], byte[]> backingStore = null; + private Map<String, Map<TopicPartition, Checkpoint>> checkpointsPerConsumerGroup; + + private volatile boolean loadSuccess = false; + private volatile boolean isInitialized = false; + + public CheckpointsStore(MirrorCheckpointTaskConfig config, Set<String> consumerGroups) { + this.config = config; + this.consumerGroups = new HashSet<>(consumerGroups); + } + + // for testing + CheckpointsStore(Map<String, Map<TopicPartition, Checkpoint>> 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<String, Map<TopicPartition, Checkpoint>> contents() { Review Comment: This is bad API practice and breaks encapsulation. The places that use contents() should be changed so that this class can accept and emit immutable objects. ########## 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<String> consumerGroups; + + private TopicAdmin cpAdmin = null; + private KafkaBasedLog<byte[], byte[]> backingStore = null; + private Map<String, Map<TopicPartition, Checkpoint>> checkpointsPerConsumerGroup; + + private volatile boolean loadSuccess = false; + private volatile boolean isInitialized = false; + + public CheckpointsStore(MirrorCheckpointTaskConfig config, Set<String> consumerGroups) { + this.config = config; + this.consumerGroups = new HashSet<>(consumerGroups); + } + + // for testing + CheckpointsStore(Map<String, Map<TopicPartition, Checkpoint>> 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<String, Map<TopicPartition, Checkpoint>> 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<String, Map<TopicPartition, Checkpoint>> readCheckpoints() { Review Comment: should this be private or package-private? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org