junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719825981



##########
File path: 
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.server.log.remote.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import scala.collection.Seq;
+import scala.collection.JavaConverters;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR;
+
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for 
usages of JavaConverters
+public class TopicBasedRemoteLogMetadataManagerRestartTest {
+
+    private static final int SEG_SIZE = 1024 * 1024;
+
+    private final Time time = new MockTime(1);
+    private final String logDir = 
TestUtils.tempDirectory("_rlmm_segs_").getAbsolutePath();
+
+    private TopicBasedRemoteLogMetadataManagerHarness 
remoteLogMetadataManagerHarness;
+
+    @BeforeEach
+    public void setup() {
+        // Start the cluster and initialize TopicBasedRemoteLogMetadataManager.
+        remoteLogMetadataManagerHarness = new 
TopicBasedRemoteLogMetadataManagerHarness() {
+            protected Map<String, Object> 
overrideRemoteLogMetadataManagerProps() {
+                Map<String, Object> props = new HashMap<>();
+                props.put(LOG_DIR, logDir);
+                return props;
+            }
+        };
+        remoteLogMetadataManagerHarness.initialize(Collections.emptySet(), 
true);
+    }
+
+    private void startTopicBasedRemoteLogMetadataManagerHarness(boolean 
startConsumerThread) {
+        
remoteLogMetadataManagerHarness.initializeRemoteLogMetadataManager(Collections.emptySet(),
 startConsumerThread);
+    }
+
+    @AfterEach
+    public void teardown() throws IOException {
+        if (remoteLogMetadataManagerHarness != null) {
+            remoteLogMetadataManagerHarness.close();
+        }
+    }
+
+    private void stopTopicBasedRemoteLogMetadataManagerHarness() throws 
IOException {
+        remoteLogMetadataManagerHarness.closeRemoteLogMetadataManager();
+    }
+
+    public TopicBasedRemoteLogMetadataManager topicBasedRlmm() {
+        return remoteLogMetadataManagerHarness.remoteLogMetadataManager();
+    }
+
+    @Test
+    public void testRLMMAPIsAfterRestart() throws Exception {
+        // Create topics.
+        String leaderTopic = "new-leader";
+        HashMap<Object, Seq<Object>> assignedLeaderTopicReplicas = new 
HashMap<>();

Review comment:
       Should we use `HashMap<Integer, Seq<Integer>>`?

##########
File path: 
storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,77 +161,73 @@ public void 
updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metada
                 throw new IllegalArgumentException("metadataUpdate: " + 
metadataUpdate + " with state " + RemoteLogSegmentState.COPY_SEGMENT_STARTED +
                                                    " can not be updated");
             case COPY_SEGMENT_FINISHED:
-                handleSegmentWithCopySegmentFinishedState(metadataUpdate, 
existingMetadata);
+                
handleSegmentWithCopySegmentFinishedState(existingMetadata.createWithUpdates(metadataUpdate));
                 break;
             case DELETE_SEGMENT_STARTED:
-                handleSegmentWithDeleteSegmentStartedState(metadataUpdate, 
existingMetadata);
+                
handleSegmentWithDeleteSegmentStartedState(existingMetadata.createWithUpdates(metadataUpdate));
                 break;
             case DELETE_SEGMENT_FINISHED:
-                handleSegmentWithDeleteSegmentFinishedState(metadataUpdate, 
existingMetadata);
+                
handleSegmentWithDeleteSegmentFinishedState(existingMetadata.createWithUpdates(metadataUpdate));
                 break;
             default:
                 throw new IllegalArgumentException("Metadata with the state " 
+ targetState + " is not supported");
         }
     }
 
-    private void 
handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadataUpdate 
metadataUpdate,
-                                                           
RemoteLogSegmentMetadata existingMetadata) {
-        log.debug("Adding remote log segment metadata to leader epoch mappings 
with update: [{}]", metadataUpdate);
-
-        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
-                
RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState);
+    protected final void 
handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata 
remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,
+                                                      (leaderEpoch, 
remoteLogLeaderEpochState, startOffset, segmentId) -> {
+                                                          long 
leaderEpochEndOffset = highestOffsetForEpoch(leaderEpoch,
+                                                                               
                             remoteLogSegmentMetadata);
+                                                          
remoteLogLeaderEpochState.handleSegmentWithCopySegmentFinishedState(startOffset,
+                                                                               
                                               segmentId,
+                                                                               
                                               leaderEpochEndOffset);
+                                                      });
 
         // Put the entry with the updated metadata.
-        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
-                existingMetadata.createWithUpdates(metadataUpdate));
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), 
remoteLogSegmentMetadata);
     }
 
-    private void 
handleSegmentWithDeleteSegmentStartedState(RemoteLogSegmentMetadataUpdate 
metadataUpdate,
-                                                            
RemoteLogSegmentMetadata existingMetadata) {
-        log.debug("Cleaning up the state for : [{}]", metadataUpdate);
+    protected final void 
handleSegmentWithDeleteSegmentStartedState(RemoteLogSegmentMetadata 
remoteLogSegmentMetadata) {
+        log.debug("Cleaning up the state for : [{}]", 
remoteLogSegmentMetadata);
 
-        doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata,
-                
RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentStartedState);
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,
+                                                      (leaderEpoch, 
remoteLogLeaderEpochState, startOffset, segmentId) ->
+                                                              
remoteLogLeaderEpochState.handleSegmentWithDeleteSegmentStartedState(startOffset,
 segmentId));
 
         // Put the entry with the updated metadata.
-        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
-                existingMetadata.createWithUpdates(metadataUpdate));
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), 
remoteLogSegmentMetadata);

Review comment:
       Got it. You have a comment that leaderEpochEntries will be removed in a 
later jira.

##########
File path: 
storage/src/main/resources/message/RemoteLogSegmentMetadataSnapshot.json
##########
@@ -0,0 +1,92 @@
+// 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.
+
+{
+  "apiKey": 3,
+  "type": "metadata",
+  "name": "RemoteLogSegmentMetadataSnapshotRecord",

Review comment:
       To be consistent, should the file name be 
RemoteLogSegmentMetadataSnapshotRecord.json?

##########
File path: 
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.server.log.remote.metadata.storage;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import scala.collection.Seq;
+import scala.collection.JavaConverters;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR;
+
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for 
usages of JavaConverters
+public class TopicBasedRemoteLogMetadataManagerRestartTest {
+
+    private static final int SEG_SIZE = 1024 * 1024;
+
+    private final Time time = new MockTime(1);
+    private final String logDir = 
TestUtils.tempDirectory("_rlmm_segs_").getAbsolutePath();
+
+    private TopicBasedRemoteLogMetadataManagerHarness 
remoteLogMetadataManagerHarness;
+
+    @BeforeEach
+    public void setup() {
+        // Start the cluster and initialize TopicBasedRemoteLogMetadataManager.
+        remoteLogMetadataManagerHarness = new 
TopicBasedRemoteLogMetadataManagerHarness() {
+            protected Map<String, Object> 
overrideRemoteLogMetadataManagerProps() {
+                Map<String, Object> props = new HashMap<>();
+                props.put(LOG_DIR, logDir);
+                return props;
+            }
+        };
+        remoteLogMetadataManagerHarness.initialize(Collections.emptySet(), 
true);
+    }
+
+    private void startTopicBasedRemoteLogMetadataManagerHarness(boolean 
startConsumerThread) {
+        
remoteLogMetadataManagerHarness.initializeRemoteLogMetadataManager(Collections.emptySet(),
 startConsumerThread);
+    }
+
+    @AfterEach
+    public void teardown() throws IOException {
+        if (remoteLogMetadataManagerHarness != null) {
+            remoteLogMetadataManagerHarness.close();
+        }
+    }
+
+    private void stopTopicBasedRemoteLogMetadataManagerHarness() throws 
IOException {
+        remoteLogMetadataManagerHarness.closeRemoteLogMetadataManager();
+    }
+
+    public TopicBasedRemoteLogMetadataManager topicBasedRlmm() {
+        return remoteLogMetadataManagerHarness.remoteLogMetadataManager();
+    }
+
+    @Test
+    public void testRLMMAPIsAfterRestart() throws Exception {
+        // Create topics.
+        String leaderTopic = "new-leader";
+        HashMap<Object, Seq<Object>> assignedLeaderTopicReplicas = new 
HashMap<>();
+        List<Object> leaderTopicReplicas = new ArrayList<>();

Review comment:
       Should we use` List<Integer>`?

##########
File path: 
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFileTest.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.server.log.remote.metadata.storage;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+
+public class RemoteLogMetadataSnapshotFileTest {
+
+    @Test
+    public void testEmptyCommittedLogMetadataFile() throws Exception {
+        File metadataStoreDir = TestUtils.tempDirectory("_rlmm_committed");
+        RemoteLogMetadataSnapshotFile snapshotFile = new 
RemoteLogMetadataSnapshotFile(metadataStoreDir.toPath());
+
+        // There should be an empty snapshot as nothing is written into it.
+        Assertions.assertFalse(snapshotFile.read().isPresent());

Review comment:
       It's easier to understand if we assertTrue on isEmpty.

##########
File path: 
storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -82,44 +87,148 @@
     // User topic partitions that this broker is a leader/follower for.
     private Set<TopicIdPartition> assignedTopicPartitions = 
Collections.emptySet();
 
-    // Map of remote log metadata topic partition to consumed offsets.
+    // Map of remote log metadata topic partition to consumed offsets. 
Received consumer records
+    // may or may not have been processed based on the assigned topic 
partitions.
     private final Map<Integer, Long> partitionToConsumedOffsets = new 
ConcurrentHashMap<>();
 
+    // Map of remote log metadata topic partition to processed offsets. 
Received consumer record is
+    // processed as the remote log metadata record's topic partition exists in 
assigned topic partitions.
+    private final Map<Integer, Long> partitionToProcessedOffsets = new 
ConcurrentHashMap<>();
+
+    // Map of remote log metadata topic partition to processed offsets that 
were synced in committedOffsetsFile.
+    private Map<Integer, Long> lastSyncedPartitionToConsumedOffsets = 
Collections.emptyMap();
+
+    private final long committedOffsetSyncIntervalMs;
+    private CommittedOffsetsFile committedOffsetsFile;
+    private long lastSyncedTimeMs;
+
     public ConsumerTask(KafkaConsumer<byte[], byte[]> consumer,
                         RemotePartitionMetadataEventHandler 
remotePartitionMetadataEventHandler,
-                        RemoteLogMetadataTopicPartitioner topicPartitioner) {
-        Objects.requireNonNull(consumer);
-        Objects.requireNonNull(remotePartitionMetadataEventHandler);
-        Objects.requireNonNull(topicPartitioner);
-
-        this.consumer = consumer;
-        this.remotePartitionMetadataEventHandler = 
remotePartitionMetadataEventHandler;
-        this.topicPartitioner = topicPartitioner;
+                        RemoteLogMetadataTopicPartitioner topicPartitioner,
+                        Path committedOffsetsPath,
+                        Time time,
+                        long committedOffsetSyncIntervalMs) {
+        this.consumer = Objects.requireNonNull(consumer);
+        this.remotePartitionMetadataEventHandler = 
Objects.requireNonNull(remotePartitionMetadataEventHandler);
+        this.topicPartitioner = Objects.requireNonNull(topicPartitioner);
+        this.time = Objects.requireNonNull(time);
+        this.committedOffsetSyncIntervalMs = committedOffsetSyncIntervalMs;
+
+        initializeConsumerAssignment(committedOffsetsPath);
+    }
+
+    private void initializeConsumerAssignment(Path committedOffsetsPath) {
+        try {
+            committedOffsetsFile = new 
CommittedOffsetsFile(committedOffsetsPath.toFile());
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+
+        Map<Integer, Long> committedOffsets = Collections.emptyMap();
+        try {
+            // Load committed offset and assign them in the consumer.
+            committedOffsets = committedOffsetsFile.readEntries();
+        } catch (IOException e) {
+            // Ignore the error and consumer consumes from the earliest offset.
+            log.error("Encountered error while building committed offsets from 
the file", e);
+        }
+
+        if (!committedOffsets.isEmpty()) {
+            // Assign topic partitions from the earlier committed offsets file.
+            Set<Integer> earlierAssignedPartitions = committedOffsets.keySet();
+            assignedMetaPartitions = 
Collections.unmodifiableSet(earlierAssignedPartitions);
+            Set<TopicPartition> metadataTopicPartitions = 
earlierAssignedPartitions.stream()
+                                                                               
    .map(x -> new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, x))
+                                                                               
    .collect(Collectors.toSet());
+            consumer.assign(metadataTopicPartitions);
+
+            // Seek to the committed offsets
+            for (Map.Entry<Integer, Long> entry : committedOffsets.entrySet()) 
{
+                partitionToConsumedOffsets.put(entry.getKey(), 
entry.getValue());
+                partitionToProcessedOffsets.put(entry.getKey(), 
entry.getValue());
+                consumer.seek(new 
TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, entry.getKey()), 
entry.getValue());
+            }
+
+            lastSyncedPartitionToConsumedOffsets = 
Collections.unmodifiableMap(committedOffsets);
+        }
     }
 
     @Override
     public void run() {
         log.info("Started Consumer task thread.");
+        lastSyncedTimeMs = time.milliseconds();
         try {
             while (!closing) {
                 maybeWaitForPartitionsAssignment();
 
                 log.info("Polling consumer to receive remote log metadata 
topic records");
-                ConsumerRecords<byte[], byte[]> consumerRecords
-                        = consumer.poll(Duration.ofMillis(POLL_INTERVAL_MS));
+                ConsumerRecords<byte[], byte[]> consumerRecords = 
consumer.poll(Duration.ofMillis(POLL_INTERVAL_MS));
                 for (ConsumerRecord<byte[], byte[]> record : consumerRecords) {
-                    handleRemoteLogMetadata(serde.deserialize(record.value()));
-                    partitionToConsumedOffsets.put(record.partition(), 
record.offset());
+                    processConsumerRecord(record);
                 }
+
+                maybeSyncCommittedDataAndOffsets(false);
             }
         } catch (Exception e) {
             log.error("Error occurred in consumer task, close:[{}]", closing, 
e);
         } finally {
+            maybeSyncCommittedDataAndOffsets(true);
             closeConsumer();
             log.info("Exiting from consumer task thread");
         }
     }
 
+    private void processConsumerRecord(ConsumerRecord<byte[], byte[]> record) {
+        // Taking assignPartitionsLock here as updateAssignmentsForPartitions 
changes assignedTopicPartitions
+        // and also calls 
remotePartitionMetadataEventHandler.clearTopicPartition(removedPartition) for 
the removed
+        // partitions.
+        RemoteLogMetadata remoteLogMetadata = 
serde.deserialize(record.value());
+        synchronized (assignPartitionsLock) {
+            if 
(assignedTopicPartitions.contains(remoteLogMetadata.topicIdPartition())) {
+                
remotePartitionMetadataEventHandler.handleRemoteLogMetadata(remoteLogMetadata);
+                partitionToProcessedOffsets.put(record.partition(), 
record.offset());

Review comment:
       Maintaining partitionToProcessedOffsets here seems a bit weird since 
it's not exactly the processed offset for a particular partition. Could we pass 
along the record offset to 
remotePartitionMetadataEventHandler.handleRemoteLogMetadata() and let 
RemotePartitionMetadataStore remember the last processed offset?




-- 
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


Reply via email to