You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/07/15 06:02:11 UTC

[GitHub] [kafka] satishd opened a new pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

satishd opened a new pull request #11058:
URL: https://github.com/apache/kafka/pull/11058


   - Added snapshots for consumed remote log metadata for each partition to avoid consuming again in case of broker restarts. These snapshots are stored in the respective topic partition log directories.
   - Added TopicBasedRemoteLogMetadataManagerRestartTest:
     * loads the earlier saved snapshots after restart
     * checks the entries are available
     * starts the consumer and add more metadata entries
     * checks the newly added entries and loaded entries are available
    
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714972637



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,21 +90,68 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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);
+        }
+
+        final Set<Map.Entry<Integer, Long>> entries = committedOffsets.entrySet();

Review comment:
       We want to go through the entries instead of calling get(key) with all the keys avoiding the lookups. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714975955



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>

Review comment:
       This was suggested in KIP review to add this as it will be useful for debugging. To make sure that, what is stored in the file and the received topic id should be the same.
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711985629



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition,
+                                                                            partitionToConsumedOffsets.get(metadataPartition));
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+            committedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);

Review comment:
       Right, it can avoid writing if it is already synced and offset did not move further. It is addressed in the latest commit.
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r725300005



##########
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:
       Since the processOffset in the checkpoint file is not really being used for now and we plan to get rid of it in the future, could we just use consumed offset for now?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719225490



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>

Review comment:
       I removed it as suggested.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711986569



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +162,46 @@ 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,
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,

Review comment:
       It is not a bug. Code is very similar but it is a minor refactoring to pass the updated RemoteLogSegmentMetadata instance.
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r723456248



##########
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());
+            } else {
+                log.debug("This event {} is skipped as the topic partition is not assigned for this instance.", remoteLogMetadata);
+            }
+            partitionToConsumedOffsets.put(record.partition(), record.offset());
+        }
+    }
+
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        // Return immediately if there is no consumption from last time.
+        boolean noConsumedOffsetUpdates = partitionToConsumedOffsets.equals(lastSyncedPartitionToConsumedOffsets);
+        if (noConsumedOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noConsumedOffsetUpdates: {}, forceSync: {}", noConsumedOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // Need to take lock on assignPartitionsLock as assignedTopicPartitions might
+            // get updated by other threads.
+            synchronized (assignPartitionsLock) {
+                for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                    int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                    Long offset = partitionToProcessedOffsets.get(metadataPartition);
+                    if (offset != null) {
+                        remotePartitionMetadataEventHandler.syncLogMetadataSnapshot(topicIdPartition, metadataPartition, offset);
+                    } else {
+                        log.debug("Skipping syncup of the remote-log-metadata-file for partition:{} , with remote log metadata partition{},  and no offset",
+                                topicIdPartition, metadataPartition);
+                    }
+                }
+
+                // Write partitionToConsumedOffsets into committed offsets file as we do not want to process them again
+                // in case of restarts.
+                committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+                lastSyncedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);
+            }
+
+            lastSyncedTimeMs = time.milliseconds();
+        } catch (IOException e) {

Review comment:
       Good point. As discussed in syncup call, we will address it in a followup PR. Filed [KAFKA-13355](  https://issues.apache.org/jira/browse/KAFKA-13355) to track this issue.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719511498



##########
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:
       `remoteLogLeaderEpochState.handleSegmentWithDeleteSegmentStartedState(startOffset, segmentId)` call in the earlier statement removes the entries. Let me know if I am missing something here. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719950502



##########
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:
       I wrote like that initially and hit the below Scala quirk before it is changed to `HashMap<Object, Seq<Object>>`, 
   ```
   /Users/satishd/repos/kafka/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java:97: error: incompatible types: inference variable A has incompatible equality constraints Object,Integer
           remoteLogMetadataManagerHarness.createTopic(leaderTopic, JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas));
                                          ^
     where A,B are type-variables:
       A extends Object declared in method <A,B>mapAsScalaMap(java.util.Map<A,B>)
       B extends Object declared in method <A,B>mapAsScalaMap(java.util.Map<A,B>)
   ``` 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-935124023


   Thanks @junrao for the comments, added in-line replies. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r717821546



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +171,37 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // partitionToConsumedOffsets is not getting changed concurrently as this method is called from #run() which updates the same.
+            // Need to take lock on assignPartitionsLock as assignedTopicPartitions might get updated by other threads.
+            synchronized (assignPartitionsLock) {
+                for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                    int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                    Long offset = partitionToConsumedOffsets.get(metadataPartition);
+                    if (offset != null && !offset.equals(committedPartitionToConsumedOffsets.get(metadataPartition))) {

Review comment:
       This means that if there is no change to a remoteLogMetadataCache, but there is new record for other partitions in the same metadataPartition, we still need to flush remoteLogMetadataCache.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,32 +90,78 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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());

Review comment:
       If the metadataTopicPartitions changes, should we remove unneeded partitions from partitionToConsumedOffsets?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -42,21 +45,36 @@
 public class RemotePartitionMetadataStore extends RemotePartitionMetadataEventHandler implements Closeable {
     private static final Logger log = LoggerFactory.getLogger(RemotePartitionMetadataStore.class);
 
+    private final Path logDir;
+
     private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
             new ConcurrentHashMap<>();
 
-    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache =
+    private Map<TopicIdPartition, FileBasedRemoteLogMetadataCache> idToRemoteLogMetadataCache =
             new ConcurrentHashMap<>();
 
+    public RemotePartitionMetadataStore(Path logDir) {
+        this.logDir = logDir;
+    }
+
     @Override
     public void handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
         log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
 
-        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        final RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+
+        // This should have been already existing as it is loaded when the partitions are assigned.
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache != null) {
+            remoteLogMetadataCache.addCopyInProgressSegment(remoteLogSegmentMetadata);
+        } else {
+            log.warn("No partition metadata found for : " + topicIdPartition);

Review comment:
       If this is unexpected, should we throw an IllegalStateException?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java
##########
@@ -39,6 +41,7 @@
     private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey();
     private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey();
     private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey();
+    private static final short REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT_API_KEY = new RemoteLogSegmentMetadataSnapshotRecord().apiKey();

Review comment:
       Hmm, since the remote log snapshot file has a file level header, having the frame header at record level seems redundant. But for simplicity, we probably could just write the framed record to the snapshot file. Could we update the comment accordingly?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -240,6 +323,11 @@ public void close() {
                 // if the closing is already set.
                 closing = true;
                 consumer.wakeup();
+                try {
+                    maybeSyncCommittedDataAndOffsets(true);

Review comment:
       Should we wait until the consumer task completes before writing the checkpoint file? Otherwise, we could be reading the in-memory state while it's being updated.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30

Review comment:
       Could we describe the format of the rest of the content too?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentMetadataSnapshot.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+public class RemoteLogSegmentMetadataSnapshot extends RemoteLogMetadata {
+
+    /**
+     * Universally unique remote log segment id.

Review comment:
       I am a bit confused. This class seems to be the same as RemoteLogSegmentMetadata?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentMetadataSnapshot.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+public class RemoteLogSegmentMetadataSnapshot extends RemoteLogMetadata {

Review comment:
       Could we add a comment for this class?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".tmp");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId().getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId().getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());

Review comment:
       Should we force the channel at the end?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,32 +90,78 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = Collections.emptyMap();

Review comment:
       To be consistent with the naming of partitionToConsumedOffsets, would it be better to name this partitionToCommittedOffsets?

##########
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:
       This is an existing issue. When removing old segments, should we remove entries from leaderEpochEntries too? 

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>

Review comment:
       The thing is that we already store topicId in a partitionMetadata file (https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers#KIP516:TopicIdentifiers-PartitionMetadatafile). Storing the same info in another file seems to add confusion.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       Since we flush the consumer offset file after flushing the snapshot file, it's possible for messages to be replayed on broker restart. Should we use metadataPartitionOffset to avoid duplicated messages being reapplied to the remote log segment metadata cache?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719951352



##########
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:
       Similar Scala quirk as I mentioned in my earlier comment.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-939474720


   Thanks @junrao for your comment, Addressed with the latest commit. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714975590



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java
##########
@@ -39,6 +41,7 @@
     private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey();
     private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey();
     private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey();
+    private static final short REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT_API_KEY = new RemoteLogSegmentMetadataSnapshotRecord().apiKey();

Review comment:
       There are two ways to implement it. 
   - Have a separate BytesApiMessageSerde for this message only and write the supporting classes which will be similar to RemoteLogMetadataSerde and RemoteLogSegmentMetadataSnapshotTransform.
   - This is one more ApiMessage about representing remote log metadata. Add to the existing RemoteLogMetadatSerde which has the framework to add one more api message. 
   
   I choose the latter for simplicity. We can update the javadoc to describe that RemoteLogMetadatSerde includes serde for all the APIMessage defined for remote log metadata including RemoteLogSegmentMetadataSnapshot. It can be used as serde for the topic as it supports all the messages stored in the remote log metadata topic. 
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-922748027


   @junrao : Thanks for the review comments. Sorry about the wrong branch merge which caused the confusion here. Addressed the comments in the latest commit. 
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r712009321



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -42,21 +45,36 @@
 public class RemotePartitionMetadataStore extends RemotePartitionMetadataEventHandler implements Closeable {
     private static final Logger log = LoggerFactory.getLogger(RemotePartitionMetadataStore.class);
 
+    private final Path logDir;
+
     private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
             new ConcurrentHashMap<>();
 
-    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache =
+    private Map<TopicIdPartition, FileBasedRemoteLogMetadataCache> idToRemoteLogMetadataCache =
             new ConcurrentHashMap<>();
 
+    public RemotePartitionMetadataStore(Path logDir) {
+        this.logDir = logDir;
+    }
+
     @Override
     public void handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
         log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
 
-        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        final RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+
+        // This should have been already existing as it is loaded when the partitions are assigned.
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache != null) {
+            remoteLogMetadataCache.addCopyInProgressSegment(remoteLogSegmentMetadata);
+        } else {
+            log.error("No partition metadata found for : " + topicIdPartition);

Review comment:
       Even if we throw an exception here, caller needs to handle these errors by logging them.  This can happen when a partition is deleted but an event for the same partition is consumed by the ConsumerTask.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714971600



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerManager.java
##########
@@ -49,21 +52,23 @@
 
     public ConsumerManager(TopicBasedRemoteLogMetadataManagerConfig rlmmConfig,
                            RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler,
-                           RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner,
+                           RemoteLogMetadataTopicPartitioner topicPartitioner,
                            Time time) {
         this.rlmmConfig = rlmmConfig;
         this.time = time;
 
         //Create a task to consume messages and submit the respective events to RemotePartitionMetadataEventHandler.
         KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(rlmmConfig.consumerProperties());
-        consumerTask = new ConsumerTask(consumer, remotePartitionMetadataEventHandler, rlmmTopicPartitioner);
+        Path committedOffsetsPath = new File(rlmmConfig.logDir(), COMMITTED_OFFSETS_FILE_NAME).toPath();
+        consumerTask = new ConsumerTask(consumer, remotePartitionMetadataEventHandler, topicPartitioner, committedOffsetsPath, time, 60_000L);

Review comment:
       Right, this will be added in a followup PR. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-925983449


   Thanks @ccding for the review, addressed with inline replies.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-933099724


   Thanks @junrao for the review comments. Addressed them with the latest commit and replies. 
   
   Added a scenario to verify checkpointed offsets in the [test](https://github.com/apache/kafka/pull/11058/commits/554df8c5e58f5dc14b5d1a3476f011184116a088#diff-8c57d1a1451531841bccd4de7f38b838cfa8444e0257c5097480e92e3e0fe72bR145).


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719226435



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       Yes, that can be done. I will address it in a followup PR.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711987288



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -91,6 +109,23 @@ public void handleRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata re
         }
     }
 
+    @Override
+    public void syncLogMetadataDataFile(TopicIdPartition topicIdPartition,
+                                        int metadataPartition,
+                                        Long metadataPartitionOffset) throws IOException {
+        //todo-tier write partitions
+        RemotePartitionDeleteMetadata partitionDeleteMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        if (partitionDeleteMetadata != null) {
+            log.info("Skipping syncing of metadata snapshot as remote partition [{}] is with state: [{}] ", topicIdPartition,
+                     partitionDeleteMetadata);
+        } else {

Review comment:
       RemotePartitionRemover may have the functionality of removing in a different way and revisit that when we have the RemotePartitionRemover functionality.
   For now, the cache is stored as part of the topic partition log directory and it will be deleted when the respective partition is deleted locally by a broker.
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722776127



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.JavaConverters;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+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 java.util.Optional;
+
+import static org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager.COMMITTED_OFFSETS_FILE_NAME;
+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<>();
+        // Set broker id 0 as the first entry which is taken as the leader.
+        leaderTopicReplicas.add(0);
+        leaderTopicReplicas.add(1);
+        leaderTopicReplicas.add(2);
+        assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(leaderTopic, JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas));
+
+        String followerTopic = "new-follower";
+        HashMap<Object, Seq<Object>> assignedFollowerTopicReplicas = new HashMap<>();
+        List<Object> followerTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        followerTopicReplicas.add(1);
+        followerTopicReplicas.add(2);
+        followerTopicReplicas.add(0);
+        assignedFollowerTopicReplicas.put(0, JavaConverters.asScalaBuffer(followerTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(followerTopic, JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas));
+
+        final TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0));
+        final TopicIdPartition followerTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0));
+
+        // Register these partitions to RLMM.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Add segments for these partitions but they are not available as they have not yet been subscribed.
+        RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()),
+                                                                                      0, 100, -1L, 0,
+                                                                                      time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get();
+
+        RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()),
+                                                                                        0, 100, -1L, 0,
+                                                                                        time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(followerSegmentMetadata).get();
+
+        // Stop TopicBasedRemoteLogMetadataManager only.
+        stopTopicBasedRemoteLogMetadataManagerHarness();
+
+        // Start TopicBasedRemoteLogMetadataManager but do not start consumer thread to check whether the stored metadata is
+        // loaded successfully or not.
+        startTopicBasedRemoteLogMetadataManagerHarness(false);
+
+        // Register these partitions to RLMM, which loads the respective metadata snapshots.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Check for the stored entries from the earlier run.
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(leaderSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(leaderTopicIdPartition)));
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(followerSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(followerTopicIdPartition)));
+        // Check whether the check-pointed consumer offsets are stored or not.
+        Path committedOffsetsPath = new File(logDir, COMMITTED_OFFSETS_FILE_NAME).toPath();
+        Assertions.assertTrue(committedOffsetsPath.toFile().exists());
+        CommittedOffsetsFile committedOffsetsFile = new CommittedOffsetsFile(committedOffsetsPath.toFile());
+
+        int metadataPartition1 = topicBasedRlmm().metadataPartition(leaderTopicIdPartition);
+        int metadataPartition2 = topicBasedRlmm().metadataPartition(followerTopicIdPartition);
+        Optional<Long> receivedOffsetForPartition1 = topicBasedRlmm().receivedOffsetForPartition(metadataPartition1);

Review comment:
       Since RLMM.initializeResources() is called in a separate thread, there is no guarantee that the ConsumerManager has been set up at this point. Should we put the check under waitUntil()?

##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.JavaConverters;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+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 java.util.Optional;
+
+import static org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager.COMMITTED_OFFSETS_FILE_NAME;
+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<>();
+        // Set broker id 0 as the first entry which is taken as the leader.
+        leaderTopicReplicas.add(0);
+        leaderTopicReplicas.add(1);
+        leaderTopicReplicas.add(2);
+        assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(leaderTopic, JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas));
+
+        String followerTopic = "new-follower";
+        HashMap<Object, Seq<Object>> assignedFollowerTopicReplicas = new HashMap<>();
+        List<Object> followerTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        followerTopicReplicas.add(1);
+        followerTopicReplicas.add(2);
+        followerTopicReplicas.add(0);
+        assignedFollowerTopicReplicas.put(0, JavaConverters.asScalaBuffer(followerTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(followerTopic, JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas));
+
+        final TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0));
+        final TopicIdPartition followerTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0));
+
+        // Register these partitions to RLMM.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Add segments for these partitions but they are not available as they have not yet been subscribed.
+        RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()),
+                                                                                      0, 100, -1L, 0,
+                                                                                      time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get();
+
+        RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()),
+                                                                                        0, 100, -1L, 0,
+                                                                                        time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(followerSegmentMetadata).get();
+
+        // Stop TopicBasedRemoteLogMetadataManager only.
+        stopTopicBasedRemoteLogMetadataManagerHarness();
+
+        // Start TopicBasedRemoteLogMetadataManager but do not start consumer thread to check whether the stored metadata is
+        // loaded successfully or not.
+        startTopicBasedRemoteLogMetadataManagerHarness(false);
+
+        // Register these partitions to RLMM, which loads the respective metadata snapshots.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Check for the stored entries from the earlier run.
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(leaderSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(leaderTopicIdPartition)));
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(followerSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(followerTopicIdPartition)));
+        // Check whether the check-pointed consumer offsets are stored or not.
+        Path committedOffsetsPath = new File(logDir, COMMITTED_OFFSETS_FILE_NAME).toPath();
+        Assertions.assertTrue(committedOffsetsPath.toFile().exists());
+        CommittedOffsetsFile committedOffsetsFile = new CommittedOffsetsFile(committedOffsetsPath.toFile());
+
+        int metadataPartition1 = topicBasedRlmm().metadataPartition(leaderTopicIdPartition);
+        int metadataPartition2 = topicBasedRlmm().metadataPartition(followerTopicIdPartition);
+        Optional<Long> receivedOffsetForPartition1 = topicBasedRlmm().receivedOffsetForPartition(metadataPartition1);
+        Optional<Long> receivedOffsetForPartition2 = topicBasedRlmm().receivedOffsetForPartition(metadataPartition2);
+        Assertions.assertTrue(receivedOffsetForPartition1.isPresent());
+        Assertions.assertTrue(receivedOffsetForPartition2.isPresent());
+
+        // Make sure these offsets are at least 0.
+        Assertions.assertTrue(receivedOffsetForPartition1.get() >= 0);

Review comment:
       Hmm, are we testing anything useful here since offset is always >= 0?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r712046006



##########
File path: checkstyle/import-control.xml
##########
@@ -303,6 +303,12 @@
       <allow pkg="org.apache.kafka.server.common" />
       <allow pkg="org.apache.kafka.server.log" />
       <allow pkg="org.apache.kafka.test" />
+
+      <subpackage name="remote">
+        <allow pkg="scala.collection" />
+        <allow pkg="scala.jdk" />

Review comment:
       Good point, this is not needed with the changes that we have. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722958956



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>
+
+    // header size: 2 (version) + 4 (partition num) + 8 (offset) = 14
+    private static final int HEADER_SIZE = 14;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        Path newMetadataSnapshotFilePath = new File(metadataStoreFile.getAbsolutePath() + ".tmp").toPath();
+        try (FileChannel fileChannel = FileChannel.open(newMetadataSnapshotFilePath,
+                                                        StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE)) {
+
+            // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+
+            // Write metadata partition offset
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // entry format: <entry-length><entry-bytes>
+
+                // Write entry length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write entry bytes
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+
+            fileChannel.force(true);
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFilePath, metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            List<RemoteLogSegmentMetadataSnapshot> result = new ArrayList<>();
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            int lenBufferReadCt;
+            while ((lenBufferReadCt = channel.read(lenBuffer)) > 0) {
+                lenBuffer.rewind();
+
+                if (lenBufferReadCt != lenBuffer.capacity()) {
+                    throw new IOException("Invalid amount of data read for the length of an entry, file may have been corrupted.");
+                }
+
+                // entry format: <entry-length><entry-bytes>
+
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadataSnapshot remoteLogSegmentMetadata =
+                        (RemoteLogSegmentMetadataSnapshot) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+    
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots;
+
+        public Snapshot(int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this(CURRENT_VERSION, metadataPartition, metadataPartitionOffset, remoteLogSegmentMetadataSnapshots);
+        }
+
+        public Snapshot(short version,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this.version = version;

Review comment:
       There is only one version for now. Added a guard against checking with the expected version.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-916281257


   @satishd : Could you rebase this PR? Thanks.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719960058



##########
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:
       As I mentioned in my earlier [comment](https://github.com/apache/kafka/pull/11058#discussion_r719295072), we will be passing offset and storing it as part of inmemory and snapshots. This will be done in a followup PR. With that, we do not need to maintain here. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-925982858


   Thanks @junrao for the review. Addressed them with the latest commit and comments. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714974207



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,35 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            HashMap<Integer, Long> syncedPartitionToConsumedOffsets = new HashMap<>();
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {

Review comment:
       Good point. Added the locking and updated it with a comment.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714975169



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,35 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            HashMap<Integer, Long> syncedPartitionToConsumedOffsets = new HashMap<>();
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                Long offset = partitionToConsumedOffsets.get(metadataPartition);
+                if (offset != null && !offset.equals(committedPartitionToConsumedOffsets.get(metadataPartition))) {
+                    remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition, offset);
+                    syncedPartitionToConsumedOffsets.put(metadataPartition, offset);
+                } else {
+                    log.debug("Skipping syncup of the remote-log-metadata-file for partition:{} , with remote log metadata partition{},  and offset:{} ",
+                            topicIdPartition, metadataPartition, offset);
+                }
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);

Review comment:
       We needed partitionToConsumedOffsets in the earlier check and we do not really need syncedPartitionToConsumedOffsets here. Updated to use partitionToConsumedOffsets for writing and setting committedPartitionToConsumedOffsets as partitionToConsumedOffsets.
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719224916



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java
##########
@@ -39,6 +41,7 @@
     private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey();
     private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey();
     private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey();
+    private static final short REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT_API_KEY = new RemoteLogSegmentMetadataSnapshotRecord().apiKey();

Review comment:
       I do not see any redundancy here.
   File level header contains 
    <version:short><metadata-partition:int><metadata-partition-offset:long>
   This is not repeated in each entry. Pl let me know if I am missing anything here.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714971397



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);
+            if (strings.length != 2) {
+                return Optional.empty();
+            }
+            int partition = Integer.parseInt(strings[0]);
+            long offset = Long.parseLong(strings[1]);

Review comment:
       An error will be thrown to the caller. As you suggested, it is good to catch that and return empty optional so that the caller throws an error with the details including the line. Addressed it in the latest commit. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722791508



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.JavaConverters;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+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 java.util.Optional;
+
+import static org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager.COMMITTED_OFFSETS_FILE_NAME;
+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<>();
+        // Set broker id 0 as the first entry which is taken as the leader.
+        leaderTopicReplicas.add(0);
+        leaderTopicReplicas.add(1);
+        leaderTopicReplicas.add(2);
+        assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(leaderTopic, JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas));
+
+        String followerTopic = "new-follower";
+        HashMap<Object, Seq<Object>> assignedFollowerTopicReplicas = new HashMap<>();
+        List<Object> followerTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        followerTopicReplicas.add(1);
+        followerTopicReplicas.add(2);
+        followerTopicReplicas.add(0);
+        assignedFollowerTopicReplicas.put(0, JavaConverters.asScalaBuffer(followerTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(followerTopic, JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas));
+
+        final TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0));
+        final TopicIdPartition followerTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0));
+
+        // Register these partitions to RLMM.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Add segments for these partitions but they are not available as they have not yet been subscribed.
+        RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()),
+                                                                                      0, 100, -1L, 0,
+                                                                                      time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get();
+
+        RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()),
+                                                                                        0, 100, -1L, 0,
+                                                                                        time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(followerSegmentMetadata).get();
+
+        // Stop TopicBasedRemoteLogMetadataManager only.
+        stopTopicBasedRemoteLogMetadataManagerHarness();
+
+        // Start TopicBasedRemoteLogMetadataManager but do not start consumer thread to check whether the stored metadata is
+        // loaded successfully or not.
+        startTopicBasedRemoteLogMetadataManagerHarness(false);
+
+        // Register these partitions to RLMM, which loads the respective metadata snapshots.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Check for the stored entries from the earlier run.
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(leaderSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(leaderTopicIdPartition)));
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(followerSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(followerTopicIdPartition)));
+        // Check whether the check-pointed consumer offsets are stored or not.
+        Path committedOffsetsPath = new File(logDir, COMMITTED_OFFSETS_FILE_NAME).toPath();
+        Assertions.assertTrue(committedOffsetsPath.toFile().exists());
+        CommittedOffsetsFile committedOffsetsFile = new CommittedOffsetsFile(committedOffsetsPath.toFile());
+
+        int metadataPartition1 = topicBasedRlmm().metadataPartition(leaderTopicIdPartition);
+        int metadataPartition2 = topicBasedRlmm().metadataPartition(followerTopicIdPartition);
+        Optional<Long> receivedOffsetForPartition1 = topicBasedRlmm().receivedOffsetForPartition(metadataPartition1);

Review comment:
       `remoteLogMetadataManagerHarness.initialize(Collections.emptySet(), true)` is already called in `setup()` method which waits until RLMM is initialized as you can see [here](https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java#L73). 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722911577



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>
+
+    // header size: 2 (version) + 4 (partition num) + 8 (offset) = 14
+    private static final int HEADER_SIZE = 14;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();

Review comment:
       This is to make fail fast if there is any issue in creating a file.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719282244



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,32 +90,78 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = Collections.emptyMap();

Review comment:
       This logic is slightly changed in the latest commit for easy to reason about. Below are the updated maps to track. 
   
   ```
       // 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();
   ``` 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711983996



##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",

Review comment:
       This is used while storing the snapshots. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711987062



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +162,46 @@ 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,
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,
                 RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState);
 
         // Put the entry with the updated metadata.
-        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
-                existingMetadata.createWithUpdates(metadataUpdate));
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), remoteLogSegmentMetadata);

Review comment:
       `doHandleSegmentStateTransitionForLeaderEpochs` takes `RemoteLogLeaderEpochState.Action` and invokes it. Each action implementer does not need all the arguments. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711989743



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       This is used to store it in the snapshot file.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r720277077



##########
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:
       ```
           HashMap<Integer, Seq<Integer>> assignedLeaderTopicReplicas = new HashMap<>();
           List<Integer> leaderTopicReplicas = new ArrayList<>();
           // Set broker id 0 as the first entry which is taken as the leader.
           leaderTopicReplicas.add(0);
           leaderTopicReplicas.add(1);
           leaderTopicReplicas.add(2);
           assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
           remoteLogMetadataManagerHarness.createTopic(leaderTopic,
               (scala.collection.Map<Object, Seq<Object>>) assignedLeaderTopicReplicas);
   ```
   It appears this would work, but I don't know which one this codebase prefers.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719516930



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentMetadataSnapshot.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+public class RemoteLogSegmentMetadataSnapshot extends RemoteLogMetadata {
+
+    /**
+     * Universally unique remote log segment id.

Review comment:
       Updated with the javadoc.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r723625488



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);

Review comment:
       It seems performance optimization is not a concern here, since deserialization of committed offsets file from disk happens only during initialization.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722772977



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>
+
+    // header size: 2 (version) + 4 (partition num) + 8 (offset) = 14
+    private static final int HEADER_SIZE = 14;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        Path newMetadataSnapshotFilePath = new File(metadataStoreFile.getAbsolutePath() + ".tmp").toPath();
+        try (FileChannel fileChannel = FileChannel.open(newMetadataSnapshotFilePath,
+                                                        StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE)) {
+
+            // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+
+            // Write metadata partition offset
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // entry format: <entry-length><entry-bytes>
+
+                // Write entry length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write entry bytes
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+
+            fileChannel.force(true);
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFilePath, metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {

Review comment:
       If we didn't create an empty file in the constructor, would we be needing this check for emptiness?

##########
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);

Review comment:
       Hmm, is this method the only place where we would sync the data to disk? I thought we would want to sync to disk periodically as well.

##########
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();

Review comment:
       There is `assignPartitionsLock` lock defined above. It would be useful to document above it what attributes/behavior it guards.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>

Review comment:
       Could we additionally store the # of entries in the header, so that during deserialization from disk we can pre-allocate the size of the result set?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>
+
+    // header size: 2 (version) + 4 (partition num) + 8 (offset) = 14
+    private static final int HEADER_SIZE = 14;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();

Review comment:
       Do we need to create the file here? It seems in L126 the call to `Utils.atomicMoveWithFallback` will create the target file if it did not exist earlier.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);

Review comment:
       Would `line.split(SEPARATOR)` be simpler?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);
+            if (strings.length != 2) {
+                return Optional.empty();
+            }
+
+            try {
+                return Optional.of(Utils.mkEntry(Integer.parseInt(strings[0]), Long.parseLong(strings[1])));
+            } catch (NumberFormatException e) {
+                return Optional.empty();
+            }
+
+        }
+    }
+
+    public synchronized void writeEntries(Map<Integer, Long> committedOffsets) throws IOException {
+        checkpointFile.write(committedOffsets.entrySet());
+    }
+
+    public synchronized Map<Integer, Long> readEntries() throws IOException {
+        List<Map.Entry<Integer, Long>> entries = checkpointFile.read();
+        Map<Integer, Long> partitionToOffsets = new HashMap<>(entries.size());
+        for (Map.Entry<Integer, Long> entry : entries) {

Review comment:
       Should we guard against a duplicate entry on disk (can happen only during corruption)?

##########
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());
+            } else {
+                log.debug("This event {} is skipped as the topic partition is not assigned for this instance.", remoteLogMetadata);
+            }
+            partitionToConsumedOffsets.put(record.partition(), record.offset());
+        }
+    }
+
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        // Return immediately if there is no consumption from last time.
+        boolean noConsumedOffsetUpdates = partitionToConsumedOffsets.equals(lastSyncedPartitionToConsumedOffsets);
+        if (noConsumedOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noConsumedOffsetUpdates: {}, forceSync: {}", noConsumedOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // Need to take lock on assignPartitionsLock as assignedTopicPartitions might
+            // get updated by other threads.
+            synchronized (assignPartitionsLock) {
+                for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                    int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                    Long offset = partitionToProcessedOffsets.get(metadataPartition);
+                    if (offset != null) {
+                        remotePartitionMetadataEventHandler.syncLogMetadataSnapshot(topicIdPartition, metadataPartition, offset);
+                    } else {
+                        log.debug("Skipping syncup of the remote-log-metadata-file for partition:{} , with remote log metadata partition{},  and no offset",
+                                topicIdPartition, metadataPartition);
+                    }
+                }
+
+                // Write partitionToConsumedOffsets into committed offsets file as we do not want to process them again
+                // in case of restarts.
+                committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+                lastSyncedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);
+            }
+
+            lastSyncedTimeMs = time.milliseconds();
+        } catch (IOException e) {

Review comment:
       Should we eventually shutdown the broker due to the IOException?
   IIUC, currently the exception gets logged and only the consumer thread dies.
   
   cc @junrao 

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>
+
+    // header size: 2 (version) + 4 (partition num) + 8 (offset) = 14
+    private static final int HEADER_SIZE = 14;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        Path newMetadataSnapshotFilePath = new File(metadataStoreFile.getAbsolutePath() + ".tmp").toPath();
+        try (FileChannel fileChannel = FileChannel.open(newMetadataSnapshotFilePath,
+                                                        StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE)) {
+
+            // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+
+            // Write metadata partition offset
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // entry format: <entry-length><entry-bytes>
+
+                // Write entry length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write entry bytes
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+
+            fileChannel.force(true);
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFilePath, metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            List<RemoteLogSegmentMetadataSnapshot> result = new ArrayList<>();
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            int lenBufferReadCt;
+            while ((lenBufferReadCt = channel.read(lenBuffer)) > 0) {
+                lenBuffer.rewind();
+
+                if (lenBufferReadCt != lenBuffer.capacity()) {
+                    throw new IOException("Invalid amount of data read for the length of an entry, file may have been corrupted.");
+                }
+
+                // entry format: <entry-length><entry-bytes>
+
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadataSnapshot remoteLogSegmentMetadata =
+                        (RemoteLogSegmentMetadataSnapshot) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+    
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots;
+
+        public Snapshot(int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this(CURRENT_VERSION, metadataPartition, metadataPartitionOffset, remoteLogSegmentMetadataSnapshots);
+        }
+
+        public Snapshot(short version,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this.version = version;

Review comment:
       Hmm, what happens if `version` doesn't match `CURRENT_VERSION`?

##########
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);

Review comment:
       Should the error message mention that the consumer is falling back to replay from earliest 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r707813690



##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",
+  "validVersions": "0",
+  "flexibleVersions": "none",

Review comment:
       Should we support flexible version from the beginning so that we could potentially support downgrade during future format changes?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {

Review comment:
       Should we do the same check when reading the lenBuffer?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -42,21 +45,36 @@
 public class RemotePartitionMetadataStore extends RemotePartitionMetadataEventHandler implements Closeable {
     private static final Logger log = LoggerFactory.getLogger(RemotePartitionMetadataStore.class);
 
+    private final Path logDir;
+
     private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
             new ConcurrentHashMap<>();
 
-    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache =
+    private Map<TopicIdPartition, FileBasedRemoteLogMetadataCache> idToRemoteLogMetadataCache =
             new ConcurrentHashMap<>();
 
+    public RemotePartitionMetadataStore(Path logDir) {
+        this.logDir = logDir;
+    }
+
     @Override
     public void handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
         log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
 
-        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        final RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+
+        // This should have been already existing as it is loaded when the partitions are assigned.
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache != null) {
+            remoteLogMetadataCache.addCopyInProgressSegment(remoteLogSegmentMetadata);
+        } else {
+            log.error("No partition metadata found for : " + topicIdPartition);

Review comment:
       Should we throw an exception?

##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    {
+      "name": "SegmentId",
+      "type": "uuid",
+      "versions": "0+",
+      "about": "Unique identifier of the log segment"
+    },
+    {
+      "name": "StartOffset",
+      "type": "int64",
+      "versions": "0+",
+      "about": "Start offset  of the segment."
+    },
+    {
+      "name": "EndOffset",
+      "type": "int64",
+      "versions": "0+",
+      "about": "End offset  of the segment."
+    },
+    {
+      "name": "BrokerId",
+      "type": "int32",
+      "versions": "0+",
+      "about": "Broker (controller or leader) id from which this event is created. DELETE_PARTITION_MARKED is sent by the controller. DELETE_PARTITION_STARTED and DELETE_PARTITION_FINISHED are sent by remote log metadata topic partition leader."
+    },
+    {
+      "name": "MaxTimestamp",
+      "type": "int64",
+      "versions": "0+",
+      "about": "Maximum timestamp with in this segment."
+    },
+    {
+      "name": "EventTimestamp",
+      "type": "int64",
+      "versions": "0+",
+      "about": "Event timestamp of this segment."
+    },
+    {
+      "name": "SegmentLeaderEpochs",
+      "type": "[]SegmentLeaderEpochEntry",
+      "versions": "0+",
+      "about": "Event timestamp of this segment.",

Review comment:
       This description of the field seems incorrect.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition,
+                                                                            partitionToConsumedOffsets.get(metadataPartition));
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+            committedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);

Review comment:
       Hmm, why do we want to reset committedPartitionToConsumedOffsets? This means if there is no new data consumed, we will still write the checkpoint file?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,21 +91,67 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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();

Review comment:
       Should we initialize committedPartitionToConsumedOffsets to the offsets in the checkpoint file?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -66,6 +70,8 @@
     private final KafkaConsumer<byte[], byte[]> consumer;
     private final RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler;
     private final RemoteLogMetadataTopicPartitioner topicPartitioner;
+    private final Time time;
+

Review comment:
       extra newline.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;

Review comment:
       remoteLogMetadatas is weird since data is the plural form of datum.

##########
File path: storage/src/main/resources/message/RemotePartitionDleteMetadataSnapshot.json
##########
@@ -0,0 +1,42 @@
+// 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": 1,
+  "type": "data",
+  "name": "RemotePartitionDeleteMetadataSnapshot",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    {
+      "name": "BrokerId",
+      "type": "int32",
+      "versions": "0+",
+      "about": "Broker (controller or leader) id from which this event is created. DELETE_PARTITION_MARKED is sent by the controller. DELETE_PARTITION_STARTED and DELETE_PARTITION_FINISHED are sent by remote log metadata topic partition leader."
+    },
+    {
+      "name": "EventTimestamp",
+      "type": "int64",
+      "versions": "0+",
+      "about": "Event timestamp of this segment."

Review comment:
       This is not about a segment.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");

Review comment:
       Our convention is to create a .tmp file.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -177,8 +254,9 @@ private void handleRemoteLogMetadata(RemoteLogMetadata remoteLogMetadata) {
 
     private void executeReassignment(Set<Integer> assignedMetaPartitionsSnapshot) {
         Set<TopicPartition> assignedMetaTopicPartitions = assignedMetaPartitionsSnapshot.stream()
-                .map(partitionNum -> new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, partitionNum))
-                .collect(Collectors.toSet());
+                                                                                        .map(partitionNum -> new TopicPartition(

Review comment:
       The line is quite long. Could we move assignedMetaPartitionsSnapshot to a new line?

##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    {
+      "name": "SegmentId",
+      "type": "uuid",
+      "versions": "0+",
+      "about": "Unique identifier of the log segment"
+    },
+    {
+      "name": "StartOffset",
+      "type": "int64",
+      "versions": "0+",
+      "about": "Start offset  of the segment."
+    },
+    {
+      "name": "EndOffset",
+      "type": "int64",
+      "versions": "0+",
+      "about": "End offset  of the segment."
+    },
+    {
+      "name": "BrokerId",
+      "type": "int32",
+      "versions": "0+",
+      "about": "Broker (controller or leader) id from which this event is created. DELETE_PARTITION_MARKED is sent by the controller. DELETE_PARTITION_STARTED and DELETE_PARTITION_FINISHED are sent by remote log metadata topic partition leader."

Review comment:
       The description seems to be intended for RemotePartitionDeleteMetadataSnapshot?

##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",

Review comment:
       How is this class used?

##########
File path: checkstyle/import-control.xml
##########
@@ -303,6 +303,12 @@
       <allow pkg="org.apache.kafka.server.common" />
       <allow pkg="org.apache.kafka.server.log" />
       <allow pkg="org.apache.kafka.test" />
+
+      <subpackage name="remote">
+        <allow pkg="scala.collection" />
+        <allow pkg="scala.jdk" />

Review comment:
       Is scala.jdk needed?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {

Review comment:
       syncCommittedDataAndOffsets => maybeSyncCommittedDataAndOffsets ?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition,
+                                                                            partitionToConsumedOffsets.get(metadataPartition));
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+            committedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);
+            lastSyncedTimeMs = time.milliseconds();
+        } catch (IOException e) {
+            log.error("Error encountered while writing committed offsets to a local file", e);

Review comment:
       Could we integrate this with logDirFailureChannel?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file

Review comment:
       Should this todo be addressed in this PR?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -91,6 +109,23 @@ public void handleRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata re
         }
     }
 
+    @Override
+    public void syncLogMetadataDataFile(TopicIdPartition topicIdPartition,
+                                        int metadataPartition,
+                                        Long metadataPartitionOffset) throws IOException {
+        //todo-tier write partitions

Review comment:
       Will the todo be addressed in this PR?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,21 +91,67 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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);
+        }
+
+        final Set<Map.Entry<Integer, Long>> entries = committedOffsets.entrySet();
+
+        if (!entries.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 : entries) {
+                partitionToConsumedOffsets.put(entry.getKey(), entry.getValue());
+                consumer.seek(new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, entry.getKey()), entry.getValue());
+            }
+

Review comment:
       extra new line.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +162,46 @@ 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,
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,

Review comment:
       So, the current code has a bug by using existingMetadata?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -91,6 +109,23 @@ public void handleRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata re
         }
     }
 
+    @Override
+    public void syncLogMetadataDataFile(TopicIdPartition topicIdPartition,
+                                        int metadataPartition,
+                                        Long metadataPartitionOffset) throws IOException {
+        //todo-tier write partitions
+        RemotePartitionDeleteMetadata partitionDeleteMetadata = idToPartitionDeleteMetadata.get(topicIdPartition);
+        if (partitionDeleteMetadata != null) {
+            log.info("Skipping syncing of metadata snapshot as remote partition [{}] is with state: [{}] ", topicIdPartition,
+                     partitionDeleteMetadata);
+        } else {

Review comment:
       Hmm, until the partition deletion completes, it seems that we still want to persist the segment list so that we could finish deleting them from the remote store?

##########
File path: storage/src/main/resources/message/RemotePartitionDleteMetadataSnapshot.json
##########
@@ -0,0 +1,42 @@
+// 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": 1,
+  "type": "data",
+  "name": "RemotePartitionDeleteMetadataSnapshot",

Review comment:
       How is this class used?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java
##########
@@ -81,12 +83,21 @@
     // requests calling different methods which use the resources like producer/consumer managers.
     private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
-    private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore();
+    private RemotePartitionMetadataStore remotePartitionMetadataStore;
     private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig;
     private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner;
     private final Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>());
     private volatile boolean initializationFailed;
 
+    public TopicBasedRemoteLogMetadataManager() {

Review comment:
       This seems unused?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       This seems unused?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +162,46 @@ 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,
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,
                 RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState);
 
         // Put the entry with the updated metadata.
-        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
-                existingMetadata.createWithUpdates(metadataUpdate));
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), remoteLogSegmentMetadata);

Review comment:
       This is an existing issue. In RemoteLogLeaderEpochState, it seems that methods like handleSegmentWithDeleteSegmentStartedState() and handleSegmentWithDeleteSegmentFinishedState() have unused input params?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711982403



##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",
+  "validVersions": "0",
+  "flexibleVersions": "none",

Review comment:
       This was an old version and merged into this branch. Corrected with the latest branch.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711990585



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -91,6 +109,23 @@ public void handleRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata re
         }
     }
 
+    @Override
+    public void syncLogMetadataDataFile(TopicIdPartition topicIdPartition,
+                                        int metadataPartition,
+                                        Long metadataPartitionOffset) throws IOException {
+        //todo-tier write partitions

Review comment:
       This todo is not valid. It is already addressed in the existing code as `remoteLogMetadataCache.flushToFile(metadataPartition, metadataPartitionOffset);`. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r720277077



##########
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:
       ```
           HashMap<Integer, Seq<Integer>> assignedLeaderTopicReplicas = new HashMap<>();
           List<Integer> leaderTopicReplicas = new ArrayList<>();
           // Set broker id 0 as the first entry which is taken as the leader.
           leaderTopicReplicas.add(0);
           leaderTopicReplicas.add(1);
           leaderTopicReplicas.add(2);
           assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
           remoteLogMetadataManagerHarness.createTopic(leaderTopic,
               (scala.collection.Map<Object, Seq<Object>>) assignedLeaderTopicReplicas);
   ```
   It appears this would work, but I don't know which one this codebase prefers.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r720277077



##########
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:
       ```        HashMap<Integer, Seq<Integer>> assignedLeaderTopicReplicas = new HashMap<>();
           List<Integer> leaderTopicReplicas = new ArrayList<>();
           // Set broker id 0 as the first entry which is taken as the leader.
           leaderTopicReplicas.add(0);
           leaderTopicReplicas.add(1);
           leaderTopicReplicas.add(2);
           assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
           remoteLogMetadataManagerHarness.createTopic(leaderTopic,
               (scala.collection.Map<Object, Seq<Object>>) assignedLeaderTopicReplicas);
   ```
   It appears this would work.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719295072



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +171,37 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // partitionToConsumedOffsets is not getting changed concurrently as this method is called from #run() which updates the same.
+            // Need to take lock on assignPartitionsLock as assignedTopicPartitions might get updated by other threads.
+            synchronized (assignPartitionsLock) {
+                for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                    int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                    Long offset = partitionToConsumedOffsets.get(metadataPartition);
+                    if (offset != null && !offset.equals(committedPartitionToConsumedOffsets.get(metadataPartition))) {

Review comment:
       This is a known limitation that I want to address in the future. One possible way I was thinking is to keep track of the last processed event’s offset to be maintained in the in-memory store and in the snapshot file. If both these entries are the same then we will skip taking a new snapshot and committing it. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-881150696


   This is built on top of https://github.com/apache/kafka/pull/10579. We can review and merge once #10579 is merged. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r708590530



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);
+            if (strings.length != 2) {
+                return Optional.empty();
+            }
+            int partition = Integer.parseInt(strings[0]);
+            long offset = Long.parseLong(strings[1]);

Review comment:
       do we need to handle NumberFormatException here?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,21 +90,68 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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);
+        }
+
+        final Set<Map.Entry<Integer, Long>> entries = committedOffsets.entrySet();

Review comment:
       out of curiosity: what is the purpose of converting a map to a set of map entries?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerManager.java
##########
@@ -49,21 +52,23 @@
 
     public ConsumerManager(TopicBasedRemoteLogMetadataManagerConfig rlmmConfig,
                            RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler,
-                           RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner,
+                           RemoteLogMetadataTopicPartitioner topicPartitioner,
                            Time time) {
         this.rlmmConfig = rlmmConfig;
         this.time = time;
 
         //Create a task to consume messages and submit the respective events to RemotePartitionMetadataEventHandler.
         KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(rlmmConfig.consumerProperties());
-        consumerTask = new ConsumerTask(consumer, remotePartitionMetadataEventHandler, rlmmTopicPartitioner);
+        Path committedOffsetsPath = new File(rlmmConfig.logDir(), COMMITTED_OFFSETS_FILE_NAME).toPath();
+        consumerTask = new ConsumerTask(consumer, remotePartitionMetadataEventHandler, topicPartitioner, committedOffsetsPath, time, 60_000L);

Review comment:
       do we plan to make `60_000L` a variable/configurable in the future? like other intervals in KafkaConfig.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722794018



##########
File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.JavaConverters;
+import scala.collection.Seq;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+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 java.util.Optional;
+
+import static org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager.COMMITTED_OFFSETS_FILE_NAME;
+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<>();
+        // Set broker id 0 as the first entry which is taken as the leader.
+        leaderTopicReplicas.add(0);
+        leaderTopicReplicas.add(1);
+        leaderTopicReplicas.add(2);
+        assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(leaderTopic, JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas));
+
+        String followerTopic = "new-follower";
+        HashMap<Object, Seq<Object>> assignedFollowerTopicReplicas = new HashMap<>();
+        List<Object> followerTopicReplicas = new ArrayList<>();
+        // Set broker id 1 as the first entry which is taken as the leader.
+        followerTopicReplicas.add(1);
+        followerTopicReplicas.add(2);
+        followerTopicReplicas.add(0);
+        assignedFollowerTopicReplicas.put(0, JavaConverters.asScalaBuffer(followerTopicReplicas));
+        remoteLogMetadataManagerHarness.createTopic(followerTopic, JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas));
+
+        final TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0));
+        final TopicIdPartition followerTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0));
+
+        // Register these partitions to RLMM.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Add segments for these partitions but they are not available as they have not yet been subscribed.
+        RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()),
+                                                                                      0, 100, -1L, 0,
+                                                                                      time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get();
+
+        RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()),
+                                                                                        0, 100, -1L, 0,
+                                                                                        time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L));
+        topicBasedRlmm().addRemoteLogSegmentMetadata(followerSegmentMetadata).get();
+
+        // Stop TopicBasedRemoteLogMetadataManager only.
+        stopTopicBasedRemoteLogMetadataManagerHarness();
+
+        // Start TopicBasedRemoteLogMetadataManager but do not start consumer thread to check whether the stored metadata is
+        // loaded successfully or not.
+        startTopicBasedRemoteLogMetadataManagerHarness(false);
+
+        // Register these partitions to RLMM, which loads the respective metadata snapshots.
+        topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition));
+
+        // Check for the stored entries from the earlier run.
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(leaderSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(leaderTopicIdPartition)));
+        Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Collections.singleton(followerSegmentMetadata).iterator(),
+                                                                 topicBasedRlmm().listRemoteLogSegments(followerTopicIdPartition)));
+        // Check whether the check-pointed consumer offsets are stored or not.
+        Path committedOffsetsPath = new File(logDir, COMMITTED_OFFSETS_FILE_NAME).toPath();
+        Assertions.assertTrue(committedOffsetsPath.toFile().exists());
+        CommittedOffsetsFile committedOffsetsFile = new CommittedOffsetsFile(committedOffsetsPath.toFile());
+
+        int metadataPartition1 = topicBasedRlmm().metadataPartition(leaderTopicIdPartition);
+        int metadataPartition2 = topicBasedRlmm().metadataPartition(followerTopicIdPartition);
+        Optional<Long> receivedOffsetForPartition1 = topicBasedRlmm().receivedOffsetForPartition(metadataPartition1);
+        Optional<Long> receivedOffsetForPartition2 = topicBasedRlmm().receivedOffsetForPartition(metadataPartition2);
+        Assertions.assertTrue(receivedOffsetForPartition1.isPresent());
+        Assertions.assertTrue(receivedOffsetForPartition2.isPresent());
+
+        // Make sure these offsets are at least 0.
+        Assertions.assertTrue(receivedOffsetForPartition1.get() >= 0);

Review comment:
       If there is no entry for this partition, it will be empty and we already have a check for that.  This offset represents messages until this offset has been received by RLMM.  So, if the received offset is 0 means it received message until 0 offset. 
   This check is to guard against any future implementation like having a default entry for all the registered metadata partitions with a value < 0.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao merged pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #11058:
URL: https://github.com/apache/kafka/pull/11058


   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r670940501



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.utils.Utils;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {

Review comment:
       This file will be enhanced to use `SnapshotFile` from #11060 once it is merged. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722958575



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // File format:
+    // <header>[<entry>...]
+    // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+    // entry: <entry-length><entry-bytes>
+
+    // header size: 2 (version) + 4 (partition num) + 8 (offset) = 14
+    private static final int HEADER_SIZE = 14;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        Path newMetadataSnapshotFilePath = new File(metadataStoreFile.getAbsolutePath() + ".tmp").toPath();
+        try (FileChannel fileChannel = FileChannel.open(newMetadataSnapshotFilePath,
+                                                        StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE)) {
+
+            // header: <version:short><metadata-partition:int><metadata-partition-offset:long>
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+
+            // Write metadata partition offset
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // entry format: <entry-length><entry-bytes>
+
+                // Write entry length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write entry bytes
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+
+            fileChannel.force(true);
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFilePath, metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {

Review comment:
       No, it is not needed in that case. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r712488028



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".tmp");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId().getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId().getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            List<RemoteLogSegmentMetadataSnapshot> result = new ArrayList<>();
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            int lenBufferReadCt;
+            while ((lenBufferReadCt = channel.read(lenBuffer)) > 0) {
+                lenBuffer.rewind();
+
+                if (lenBufferReadCt != lenBuffer.capacity()) {
+                    throw new IOException("Invalid amount of data read for the length of an entry, file may have been corrupted.");
+                }
+
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadataSnapshot remoteLogSegmentMetadata =
+                        (RemoteLogSegmentMetadataSnapshot) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogSegmentMetadataSnapshots);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogSegmentMetadataSnapshots = remoteLogSegmentMetadataSnapshots;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {
+            return metadataPartitionOffset;
+        }
+
+        public Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots() {
+            return remoteLogSegmentMetadataSnapshots;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof Snapshot)) return false;
+            Snapshot snapshot = (Snapshot) o;
+            return version == snapshot.version && metadataPartition == snapshot.metadataPartition && metadataPartitionOffset == snapshot.metadataPartitionOffset && Objects
+                    .equals(topicId, snapshot.topicId);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(version, topicId, metadataPartition, metadataPartitionOffset);
+        }
+
+        @Override
+        public String toString() {
+            return "Snapshot{" +
+                    "version=" + version +
+                    ", topicId=" + topicId +
+                    ", metadataPartition=" + metadataPartition +
+                    ", metadataPartitionOffset=" + metadataPartitionOffset +

Review comment:
       Should we include some metadata (e.g. size) of remoteLogSegmentMetadataSnapshots?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,35 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            HashMap<Integer, Long> syncedPartitionToConsumedOffsets = new HashMap<>();
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {

Review comment:
       assignedTopicPartitions could be updated concurrently and we are accessing it without lock protection here.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataSnapshotRecordTransform.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.serialization;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class RemoteLogSegmentMetadataSnapshotRecordTransform implements RemoteLogMetadataTransform<RemoteLogSegmentMetadata> {

Review comment:
       This class seems never used?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java
##########
@@ -39,6 +41,7 @@
     private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey();
     private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey();
     private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey();
+    private static final short REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT_API_KEY = new RemoteLogSegmentMetadataSnapshotRecord().apiKey();

Review comment:
       REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT is not in internal remote log metadata topic and the comment of the class says this is for messages in remote log metadata topic. So, do we need this change?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentMetadataSnapshot.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+public class RemoteLogSegmentMetadataSnapshot extends RemoteLogMetadata {
+
+    /**
+     * Universally unique remote log segment id.
+     */
+    private final Uuid segmentId;
+
+    /**
+     * Start offset of this segment.
+     */
+    private final long startOffset;
+
+    /**
+     * End offset of this segment.
+     */
+    private final long endOffset;
+
+    /**
+     * Maximum timestamp in milli seconds in the segment
+     */
+    private final long maxTimestampMs;
+
+    /**
+     * LeaderEpoch vs offset for messages within this segment.
+     */
+    private final NavigableMap<Integer, Long> segmentLeaderEpochs;
+
+    /**
+     * Size of the segment in bytes.
+     */
+    private final int segmentSizeInBytes;
+
+    /**
+     * It indicates the state in which the action is executed on this segment.
+     */
+    private final RemoteLogSegmentState state;
+
+    /**
+     * Creates an instance with the given metadata of remote log segment.
+     * <p>
+     * {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
+     * then it should have an entry with epoch mapping to start-offset of this segment.
+     *
+     * @param segmentId                  Universally unique remote log segment id.
+     * @param startOffset         Start offset of this segment (inclusive).
+     * @param endOffset           End offset of this segment (inclusive).
+     * @param maxTimestampMs      Maximum timestamp in milli seconds in this segment.
+     * @param brokerId            Broker id from which this event is generated.
+     * @param eventTimestampMs    Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage.
+     * @param segmentSizeInBytes  Size of this segment in bytes.
+     * @param state               State of the respective segment of remoteLogSegmentId.
+     * @param segmentLeaderEpochs leader epochs occurred within this segment.
+     */
+    public RemoteLogSegmentMetadataSnapshot(Uuid segmentId,
+                                            long startOffset,
+                                            long endOffset,
+                                            long maxTimestampMs,
+                                            int brokerId,
+                                            long eventTimestampMs,
+                                            int segmentSizeInBytes,
+                                            RemoteLogSegmentState state,
+                                            Map<Integer, Long> segmentLeaderEpochs) {
+        super(brokerId, eventTimestampMs);
+        this.segmentId = Objects.requireNonNull(segmentId, "remoteLogSegmentId can not be null");
+        this.state = Objects.requireNonNull(state, "state can not be null");
+
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+        this.maxTimestampMs = maxTimestampMs;
+        this.segmentSizeInBytes = segmentSizeInBytes;
+
+        if (segmentLeaderEpochs == null || segmentLeaderEpochs.isEmpty()) {
+            throw new IllegalArgumentException("segmentLeaderEpochs can not be null or empty");
+        }
+
+        this.segmentLeaderEpochs = Collections.unmodifiableNavigableMap(new TreeMap<>(segmentLeaderEpochs));
+    }
+
+    public static RemoteLogSegmentMetadataSnapshot create(RemoteLogSegmentMetadata metadata) {
+        return new RemoteLogSegmentMetadataSnapshot(metadata.remoteLogSegmentId().id(), metadata.startOffset(), metadata.endOffset(),
+                                                    metadata.maxTimestampMs(), metadata.brokerId(), metadata.eventTimestampMs(),
+                                                    metadata.segmentSizeInBytes(), metadata.state(), metadata.segmentLeaderEpochs());
+    }
+
+    /**
+     * @return unique id of this segment.
+     */
+    public Uuid segmentId() {
+        return segmentId;
+    }
+
+    /**
+     * @return Start offset of this segment (inclusive).
+     */
+    public long startOffset() {
+        return startOffset;
+    }
+
+    /**
+     * @return End offset of this segment (inclusive).
+     */
+    public long endOffset() {
+        return endOffset;
+    }
+
+    /**
+     * @return Total size of this segment in bytes.
+     */
+    public int segmentSizeInBytes() {
+        return segmentSizeInBytes;
+    }
+
+    /**
+     * @return Maximum timestamp in milli seconds of a record within this segment.
+     */
+    public long maxTimestampMs() {
+        return maxTimestampMs;
+    }
+
+    /**
+     * @return Map of leader epoch vs offset for the records available in this segment.
+     */
+    public NavigableMap<Integer, Long> segmentLeaderEpochs() {
+        return segmentLeaderEpochs;
+    }
+
+    /**
+     * Returns the current state of this remote log segment. It can be any of the below
+     * <ul>
+     *     {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}
+     *     {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}
+     *     {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}
+     *     {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}
+     * </ul>
+     */
+    public RemoteLogSegmentState state() {
+        return state;
+    }
+
+    @Override
+    public TopicIdPartition topicIdPartition() {
+        throw new UnsupportedOperationException("This metadata does not have topic partition with it.");
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof RemoteLogSegmentMetadataSnapshot)) return false;
+        RemoteLogSegmentMetadataSnapshot that = (RemoteLogSegmentMetadataSnapshot) o;
+        return startOffset == that.startOffset && endOffset == that.endOffset && maxTimestampMs == that.maxTimestampMs && segmentSizeInBytes == that.segmentSizeInBytes && Objects.equals(
+                segmentId, that.segmentId) && Objects.equals(segmentLeaderEpochs, that.segmentLeaderEpochs) && state == that.state;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(segmentId, startOffset, endOffset, maxTimestampMs, segmentLeaderEpochs, segmentSizeInBytes, state);
+    }
+
+    @Override
+    public String toString() {
+        return "RemoteLogSegmentMetadataSnapshot{" +
+                "segmentId=" + segmentId +
+                ", startOffset=" + startOffset +
+                ", endOffset=" + endOffset +
+                ", maxTimestampMs=" + maxTimestampMs +
+                ", segmentSizeInBytes=" + segmentSizeInBytes +
+                ", state=" + state +

Review comment:
       Should toString() include segmentLeaderEpochs too?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,21 +90,68 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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);
+        }
+
+        final Set<Map.Entry<Integer, Long>> entries = committedOffsets.entrySet();
+
+        if (!entries.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 : entries) {
+                partitionToConsumedOffsets.put(entry.getKey(), entry.getValue());
+                consumer.seek(new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, entry.getKey()), entry.getValue());
+            }
+
+            committedPartitionToConsumedOffsets = committedOffsets;
+        }
     }
 
     @Override
     public void run() {
         log.info("Started Consumer task thread.");
+        lastSyncedTimeMs = time.milliseconds();

Review comment:
       Below, maybeWaitForPartitionsAssignment() may add new assignedMetaPartitions. We need to seek those partitions from the beginning.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       Yes, but it seems that we never read it?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".tmp");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId().getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId().getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            List<RemoteLogSegmentMetadataSnapshot> result = new ArrayList<>();
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            int lenBufferReadCt;
+            while ((lenBufferReadCt = channel.read(lenBuffer)) > 0) {
+                lenBuffer.rewind();
+
+                if (lenBufferReadCt != lenBuffer.capacity()) {
+                    throw new IOException("Invalid amount of data read for the length of an entry, file may have been corrupted.");
+                }
+
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadataSnapshot remoteLogSegmentMetadata =
+                        (RemoteLogSegmentMetadataSnapshot) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogSegmentMetadataSnapshots);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogSegmentMetadataSnapshots = remoteLogSegmentMetadataSnapshots;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {
+            return metadataPartitionOffset;
+        }
+
+        public Collection<RemoteLogSegmentMetadataSnapshot> remoteLogSegmentMetadataSnapshots() {
+            return remoteLogSegmentMetadataSnapshots;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof Snapshot)) return false;
+            Snapshot snapshot = (Snapshot) o;
+            return version == snapshot.version && metadataPartition == snapshot.metadataPartition && metadataPartitionOffset == snapshot.metadataPartitionOffset && Objects
+                    .equals(topicId, snapshot.topicId);

Review comment:
       Should we include remoteLogSegmentMetadataSnapshots too? 

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +162,46 @@ 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,
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,

Review comment:
       The param in doHandleSegmentStateTransitionForLeaderEpochs() is still called existingMetadata. Could we update that accordingly?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +162,46 @@ 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,
+    private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        doHandleSegmentStateTransitionForLeaderEpochs(remoteLogSegmentMetadata,
                 RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState);
 
         // Put the entry with the updated metadata.
-        idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(),
-                existingMetadata.createWithUpdates(metadataUpdate));
+        idToSegmentMetadata.put(remoteLogSegmentMetadata.remoteLogSegmentId(), remoteLogSegmentMetadata);

Review comment:
       It seems that leaderEpochEndOffset is never used below.
   
   ```
       void handleSegmentWithDeleteSegmentStartedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId,
                                                       Long leaderEpochEndOffset) {
   
   ```
   
   It seems that startOffset and leaderEpochEndOffset are not used in the follow method.
   ```
       void handleSegmentWithDeleteSegmentFinishedState(long startOffset, RemoteLogSegmentId remoteLogSegmentId,
                                                        Long leaderEpochEndOffset) {
   
   ```

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,35 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            HashMap<Integer, Long> syncedPartitionToConsumedOffsets = new HashMap<>();
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                Long offset = partitionToConsumedOffsets.get(metadataPartition);
+                if (offset != null && !offset.equals(committedPartitionToConsumedOffsets.get(metadataPartition))) {
+                    remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition, offset);
+                    syncedPartitionToConsumedOffsets.put(metadataPartition, offset);
+                } else {
+                    log.debug("Skipping syncup of the remote-log-metadata-file for partition:{} , with remote log metadata partition{},  and offset:{} ",
+                            topicIdPartition, metadataPartition, offset);
+                }
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);

Review comment:
       Some of the metadataPartitions could have been removed. Should we write syncedPartitionToConsumedOffsets instead? Should we also set partitionToConsumedOffsets to syncedPartitionToConsumedOffsets?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>

Review comment:
       Why do we need topicId since this file is already inside a topic partition?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +161,46 @@ 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,
+    protected final void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {

Review comment:
       RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState() has the logic to move the existing segment with the same offset in offsetToId to unreferencedSegmentIds, which seems sensitive to ordering. However, the snapshot are stored in an arbitrary order. Is that an issue?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition,
+                                                                            partitionToConsumedOffsets.get(metadataPartition));
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+            committedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);
+            lastSyncedTimeMs = time.milliseconds();
+        } catch (IOException e) {
+            log.error("Error encountered while writing committed offsets to a local file", e);

Review comment:
       This is an IOException from local storage. So, why do we want to wait when remote log subsystem is added? Ideally, we want to integrate it with logDirFailureChannel. If we want to do that later, it seems that we should at least throw an exception to stop the thread?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714976377



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java
##########
@@ -161,53 +161,46 @@ 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,
+    protected final void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {

Review comment:
       Good catch. We should maintain the ordering while storing the segments. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719940964



##########
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:
       [Optional](https://docs.oracle.com/javase/8/docs/api/java/util/Optional.html) does not have `isEmpty()` method in JDK 8, that's why using `isPresent ()` and the respective check. `isEmpty()` is available since JDK 11.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714976848



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       This is added more for debugging purposes. This was suggested while the KIP was reviewed.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711983996



##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataRecordSnapshot.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": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordSnapshot",

Review comment:
       This is used while storing the snapshots ser/des. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-935967523


   Thanks @kowshik for the review comments. Added inline replies and addressed with the latest commit. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-913321617


   @junrao gentle reminder to review the changes.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r702422832



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.utils.Utils;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {

Review comment:
       This is updated with the latest commit.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-931422554


   Thanks @junrao for the review. Addressed them with the replies and with the latest commit. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719512519



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -240,6 +323,11 @@ public void close() {
                 // if the closing is already set.
                 closing = true;
                 consumer.wakeup();
+                try {
+                    maybeSyncCommittedDataAndOffsets(true);

Review comment:
       Another way is to call `maybeSyncCommittedDataAndOffsets` in finally block of run method before `closeConsumer()`. This will avoid running in multiple threads and the state that is read is consistent. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-893543468


   @junrao: Please review this PR once https://github.com/apache/kafka/pull/11060 is reviewed and merged. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#issuecomment-917351522


   @junrao Rebased the PR and resolved the conflicts, thanks.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711987740



##########
File path: storage/src/main/resources/message/RemotePartitionDleteMetadataSnapshot.json
##########
@@ -0,0 +1,42 @@
+// 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": 1,
+  "type": "data",
+  "name": "RemotePartitionDeleteMetadataSnapshot",

Review comment:
       Removed for now as it is not needed now. We will revisit this later if needed. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722884223



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);

Review comment:
       I am not sure how would that be simpler. `String.split(regex)` uses `Pattern.compile(regex).split()` for this usecase and it needs to compile regex for each invocation of fromString() method. 
   So, I decided to use `MINIMUM_ONE_WHITESPACE.split(line)` as `Pattern.compile()` is done only once, which is more efective




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711985297



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file

Review comment:
       invalid todo, this is already handled by calling remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition, 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r714973902



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,21 +90,68 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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);
+        }
+
+        final Set<Map.Entry<Integer, Long>> entries = committedOffsets.entrySet();
+
+        if (!entries.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 : entries) {
+                partitionToConsumedOffsets.put(entry.getKey(), entry.getValue());
+                consumer.seek(new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, entry.getKey()), entry.getValue());
+            }
+
+            committedPartitionToConsumedOffsets = committedOffsets;
+        }
     }
 
     @Override
     public void run() {
         log.info("Started Consumer task thread.");
+        lastSyncedTimeMs = time.milliseconds();

Review comment:
       I guess the consumer would fetch from the earliest offset as there is no state associated with in the consumer for those partitions as `auto.offset.reset` is set as `earliest`. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r712001447



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +174,29 @@ public void run() {
         }
     }
 
+    private void syncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // todo sync the snapshot file
+            for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                remotePartitionMetadataEventHandler.syncLogMetadataDataFile(topicIdPartition, metadataPartition,
+                                                                            partitionToConsumedOffsets.get(metadataPartition));
+            }
+
+            committedOffsetsFile.writeEntries(partitionToConsumedOffsets);
+            committedPartitionToConsumedOffsets = new HashMap<>(partitionToConsumedOffsets);
+            lastSyncedTimeMs = time.milliseconds();
+        } catch (IOException e) {
+            log.error("Error encountered while writing committed offsets to a local file", e);

Review comment:
       For now, we will log an error if there are any issues in writing to the file. There are no direct dependencies to `logDirFailureChannel` as it is part of RLMM implementation. We can address this in PRs when remote log subsystem is added. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711989139



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java
##########
@@ -81,12 +83,21 @@
     // requests calling different methods which use the resources like producer/consumer managers.
     private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
-    private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore();
+    private RemotePartitionMetadataStore remotePartitionMetadataStore;
     private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig;
     private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner;
     private final Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>());
     private volatile boolean initializationFailed;
 
+    public TopicBasedRemoteLogMetadataManager() {

Review comment:
       This will be used when the remote log subsystem is added and it will be used as the default RLMM if it is not configured by a user. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r711983366



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");

Review comment:
       sure, updated to use the common convention.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r717821546



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +171,37 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // partitionToConsumedOffsets is not getting changed concurrently as this method is called from #run() which updates the same.
+            // Need to take lock on assignPartitionsLock as assignedTopicPartitions might get updated by other threads.
+            synchronized (assignPartitionsLock) {
+                for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                    int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                    Long offset = partitionToConsumedOffsets.get(metadataPartition);
+                    if (offset != null && !offset.equals(committedPartitionToConsumedOffsets.get(metadataPartition))) {

Review comment:
       This means that if there is no change to a remoteLogMetadataCache, but there is new record for other partitions in the same metadataPartition, we still need to flush remoteLogMetadataCache.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,32 +90,78 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = 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());

Review comment:
       If the metadataTopicPartitions changes, should we remove unneeded partitions from partitionToConsumedOffsets?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemotePartitionMetadataStore.java
##########
@@ -42,21 +45,36 @@
 public class RemotePartitionMetadataStore extends RemotePartitionMetadataEventHandler implements Closeable {
     private static final Logger log = LoggerFactory.getLogger(RemotePartitionMetadataStore.class);
 
+    private final Path logDir;
+
     private Map<TopicIdPartition, RemotePartitionDeleteMetadata> idToPartitionDeleteMetadata =
             new ConcurrentHashMap<>();
 
-    private Map<TopicIdPartition, RemoteLogMetadataCache> idToRemoteLogMetadataCache =
+    private Map<TopicIdPartition, FileBasedRemoteLogMetadataCache> idToRemoteLogMetadataCache =
             new ConcurrentHashMap<>();
 
+    public RemotePartitionMetadataStore(Path logDir) {
+        this.logDir = logDir;
+    }
+
     @Override
     public void handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
         log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata);
 
-        RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        final RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId();
+        TopicIdPartition topicIdPartition = remoteLogSegmentId.topicIdPartition();
+
+        // This should have been already existing as it is loaded when the partitions are assigned.
+        RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition);
+        if (remoteLogMetadataCache != null) {
+            remoteLogMetadataCache.addCopyInProgressSegment(remoteLogSegmentMetadata);
+        } else {
+            log.warn("No partition metadata found for : " + topicIdPartition);

Review comment:
       If this is unexpected, should we throw an IllegalStateException?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java
##########
@@ -39,6 +41,7 @@
     private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey();
     private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey();
     private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey();
+    private static final short REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT_API_KEY = new RemoteLogSegmentMetadataSnapshotRecord().apiKey();

Review comment:
       Hmm, since the remote log snapshot file has a file level header, having the frame header at record level seems redundant. But for simplicity, we probably could just write the framed record to the snapshot file. Could we update the comment accordingly?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -240,6 +323,11 @@ public void close() {
                 // if the closing is already set.
                 closing = true;
                 consumer.wakeup();
+                try {
+                    maybeSyncCommittedDataAndOffsets(true);

Review comment:
       Should we wait until the consumer task completes before writing the checkpoint file? Otherwise, we could be reading the in-memory state while it's being updated.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30

Review comment:
       Could we describe the format of the rest of the content too?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentMetadataSnapshot.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+public class RemoteLogSegmentMetadataSnapshot extends RemoteLogMetadata {
+
+    /**
+     * Universally unique remote log segment id.

Review comment:
       I am a bit confused. This class seems to be the same as RemoteLogSegmentMetadata?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentMetadataSnapshot.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+public class RemoteLogSegmentMetadataSnapshot extends RemoteLogMetadata {

Review comment:
       Could we add a comment for this class?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+    private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".tmp");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version());
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId().getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId().getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition());
+            headerBuffer.putLong(snapshot.metadataPartitionOffset());
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            for (RemoteLogSegmentMetadataSnapshot metadataSnapshot : snapshot.remoteLogSegmentMetadataSnapshots()) {
+                final byte[] serializedBytes = serde.serialize(metadataSnapshot);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());

Review comment:
       Should we force the channel at the end?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -85,32 +90,78 @@
     // Map of remote log metadata topic partition to consumed offsets.
     private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>();
 
+    private Map<Integer, Long> committedPartitionToConsumedOffsets = Collections.emptyMap();

Review comment:
       To be consistent with the naming of partitionToConsumedOffsets, would it be better to name this partitionToCommittedOffsets?

##########
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:
       This is an existing issue. When removing old segments, should we remove entries from leaderEpochEntries too? 

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>

Review comment:
       The thing is that we already store topicId in a partitionMetadata file (https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers#KIP516:TopicIdentifiers-PartitionMetadatafile). Storing the same info in another file seems to add confusion.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSnapshotFile.java
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This class represents the remote log data snapshot stored in a file for a specific topic partition. This is used by
+ * {@link TopicBasedRemoteLogMetadataManager} to store the remote log metadata received for a specific partition from
+ * remote log metadata topic. This will avoid reading the remote log metadata messages from the topic again when a
+ * broker restarts.
+ */
+public class RemoteLogMetadataSnapshotFile {
+    private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataSnapshotFile.class);
+
+    public static final String COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME = "remote_log_snapshot";
+
+    // header: <version:short><topicId:2 longs><metadata-partition:int><metadata-partition-offset:long>
+    // size: 2 + (8+8) + 4 + 8 = 30
+    private static final int HEADER_SIZE = 30;
+
+    private final File metadataStoreFile;
+
+    /**
+     * Creates a CommittedLogMetadataSnapshotFile instance backed by a file with the name `remote_log_snapshot` in
+     * the given {@code metadataStoreDir}. It creates the file if it does not exist.
+     *
+     * @param metadataStoreDir directory in which the snapshot file to be created.
+     */
+    RemoteLogMetadataSnapshotFile(Path metadataStoreDir) {
+        this.metadataStoreFile = new File(metadataStoreDir.toFile(), COMMITTED_LOG_METADATA_SNAPSHOT_FILE_NAME);
+
+        // Create an empty file if it does not exist.
+        try {
+            boolean newFileCreated = metadataStoreFile.createNewFile();
+            log.info("Remote log metadata snapshot file: [{}], newFileCreated: [{}]", metadataStoreFile, newFileCreated);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Writes the given snapshot replacing the earlier snapshot data.
+     *
+     * @param snapshot Snapshot to be stored.
+     * @throws IOException if there4 is any error in writing the given snapshot to the file.
+     */
+    public synchronized void write(Snapshot snapshot) throws IOException {
+        File newMetadataSnapshotFile = new File(metadataStoreFile.getAbsolutePath() + ".new");
+        try (WritableByteChannel fileChannel = Channels.newChannel(new FileOutputStream(newMetadataSnapshotFile))) {
+
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+
+            // Write version
+            headerBuffer.putShort(snapshot.version);
+
+            // Write topic-id
+            headerBuffer.putLong(snapshot.topicId.getMostSignificantBits());
+            headerBuffer.putLong(snapshot.topicId.getLeastSignificantBits());
+
+            // Write metadata partition and metadata partition offset
+            headerBuffer.putInt(snapshot.metadataPartition);
+            headerBuffer.putLong(snapshot.metadataPartitionOffset);
+            headerBuffer.flip();
+
+            // Write header
+            fileChannel.write(headerBuffer);
+
+            // Write each entry
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+            for (RemoteLogSegmentMetadata remoteLogSegmentMetadata : snapshot.remoteLogMetadatas) {
+                final byte[] serializedBytes = serde.serialize(remoteLogSegmentMetadata);
+                // Write length
+                lenBuffer.putInt(serializedBytes.length);
+                lenBuffer.flip();
+                fileChannel.write(lenBuffer);
+                lenBuffer.rewind();
+
+                // Write data
+                fileChannel.write(ByteBuffer.wrap(serializedBytes));
+            }
+        }
+
+        Utils.atomicMoveWithFallback(newMetadataSnapshotFile.toPath(), metadataStoreFile.toPath());
+    }
+
+    /**
+     * @return the Snapshot if it exists.
+     * @throws IOException if there is any error in reading the stored snapshot.
+     */
+    @SuppressWarnings("unchecked")
+    public synchronized Optional<Snapshot> read() throws IOException {
+
+        // Checking for empty files.
+        if (metadataStoreFile.length() == 0) {
+            return Optional.empty();
+        }
+
+        try (ReadableByteChannel channel = Channels.newChannel(new FileInputStream(metadataStoreFile))) {
+
+            // Read header
+            ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE);
+            channel.read(headerBuffer);
+            headerBuffer.rewind();
+            short version = headerBuffer.getShort();
+            Uuid topicId = new Uuid(headerBuffer.getLong(), headerBuffer.getLong());
+            int metadataPartition = headerBuffer.getInt();
+            long metadataPartitionOffset = headerBuffer.getLong();
+
+            RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde();
+
+            List<RemoteLogSegmentMetadata> result = new ArrayList<>();
+
+            ByteBuffer lenBuffer = ByteBuffer.allocate(4);
+            while (channel.read(lenBuffer) > 0) {
+                lenBuffer.rewind();
+                // Read the length of each entry
+                final int len = lenBuffer.getInt();
+                lenBuffer.rewind();
+
+                // Read the entry
+                ByteBuffer data = ByteBuffer.allocate(len);
+                final int read = channel.read(data);
+                if (read != len) {
+                    throw new IOException("Invalid amount of data read, file may have been corrupted.");
+                }
+
+                // We are always adding RemoteLogSegmentMetadata only as you can see in #write() method.
+                // Did not add a specific serde for RemoteLogSegmentMetadata and reusing RemoteLogMetadataSerde
+                final RemoteLogSegmentMetadata remoteLogSegmentMetadata = (RemoteLogSegmentMetadata) serde.deserialize(data.array());
+                result.add(remoteLogSegmentMetadata);
+            }
+
+            return Optional.of(new Snapshot(version, topicId, metadataPartition, metadataPartitionOffset, result));
+        }
+    }
+
+    /**
+     * This class represents the collection of remote log metadata for a specific topic partition.
+     */
+    public static final class Snapshot {
+        private static final short CURRENT_VERSION = 0;
+
+        private final short version;
+        private final Uuid topicId;
+        private final int metadataPartition;
+        private final long metadataPartitionOffset;
+        private final Collection<RemoteLogSegmentMetadata> remoteLogMetadatas;
+
+        public Snapshot(Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this(CURRENT_VERSION, topicId, metadataPartition, metadataPartitionOffset, remoteLogMetadatas);
+        }
+
+        public Snapshot(short version,
+                        Uuid topicId,
+                        int metadataPartition,
+                        long metadataPartitionOffset,
+                        Collection<RemoteLogSegmentMetadata> remoteLogMetadatas) {
+            this.version = version;
+            this.topicId = topicId;
+            this.metadataPartition = metadataPartition;
+            this.metadataPartitionOffset = metadataPartitionOffset;
+            this.remoteLogMetadatas = remoteLogMetadatas;
+        }
+
+        public short version() {
+            return version;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int metadataPartition() {
+            return metadataPartition;
+        }
+
+        public long metadataPartitionOffset() {

Review comment:
       Since we flush the consumer offset file after flushing the snapshot file, it's possible for messages to be replayed on broker restart. Should we use metadataPartitionOffset to avoid duplicated messages being reapplied to the remote log segment metadata cache?




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722923763



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/CommittedOffsetsFile.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.utils.Utils;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class represents a file containing the committed offsets of remote log metadata partitions.
+ */
+public class CommittedOffsetsFile {
+    private static final int CURRENT_VERSION = 0;
+    private static final String SEPARATOR = " ";
+
+    private static final Pattern MINIMUM_ONE_WHITESPACE = Pattern.compile("\\s+");
+    private final CheckpointFile<Map.Entry<Integer, Long>> checkpointFile;
+
+    CommittedOffsetsFile(File offsetsFile) throws IOException {
+        CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> formatter = new EntryFormatter();
+        checkpointFile = new CheckpointFile<>(offsetsFile, CURRENT_VERSION, formatter);
+    }
+
+    private static class EntryFormatter implements CheckpointFile.EntryFormatter<Map.Entry<Integer, Long>> {
+
+        @Override
+        public String toString(Map.Entry<Integer, Long> entry) {
+            // Each entry is stored in a new line as <partition-num offset>
+            return entry.getKey() + SEPARATOR + entry.getValue();
+        }
+
+        @Override
+        public Optional<Map.Entry<Integer, Long>> fromString(String line) {
+            String[] strings = MINIMUM_ONE_WHITESPACE.split(line);
+            if (strings.length != 2) {
+                return Optional.empty();
+            }
+
+            try {
+                return Optional.of(Utils.mkEntry(Integer.parseInt(strings[0]), Long.parseLong(strings[1])));
+            } catch (NumberFormatException e) {
+                return Optional.empty();
+            }
+
+        }
+    }
+
+    public synchronized void writeEntries(Map<Integer, Long> committedOffsets) throws IOException {
+        checkpointFile.write(committedOffsets.entrySet());
+    }
+
+    public synchronized Map<Integer, Long> readEntries() throws IOException {
+        List<Map.Entry<Integer, Long>> entries = checkpointFile.read();
+        Map<Integer, Long> partitionToOffsets = new HashMap<>(entries.size());
+        for (Map.Entry<Integer, Long> entry : entries) {

Review comment:
       Good point. I will add a check against that. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r720277077



##########
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:
       ```        HashMap<Integer, Seq<Integer>> assignedLeaderTopicReplicas = new HashMap<>();
           List<Integer> leaderTopicReplicas = new ArrayList<>();
           // Set broker id 0 as the first entry which is taken as the leader.
           leaderTopicReplicas.add(0);
           leaderTopicReplicas.add(1);
           leaderTopicReplicas.add(2);
           assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas));
           remoteLogMetadataManagerHarness.createTopic(leaderTopic,
               (scala.collection.Map<Object, Seq<Object>>) assignedLeaderTopicReplicas);
   ```
   It appears this would work, but I don't know which one this codebase prefers.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719224916



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java
##########
@@ -39,6 +41,7 @@
     private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey();
     private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey();
     private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey();
+    private static final short REMOTE_LOG_SEGMENT_METADATA_SNAPSHOT_API_KEY = new RemoteLogSegmentMetadataSnapshotRecord().apiKey();

Review comment:
       I do not see any redundancy here.
   File level header contains 
   ` <version:short><metadata-partition:int><metadata-partition-offset:long>`
   This is not repeated in each entry. Pl let me know if I am missing anything here.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r722946986



##########
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);

Review comment:
       We wanted to do this in the same thread for now. We will do it in a separate thread if it is really needed.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719512519



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -240,6 +323,11 @@ public void close() {
                 // if the closing is already set.
                 closing = true;
                 consumer.wakeup();
+                try {
+                    maybeSyncCommittedDataAndOffsets(true);

Review comment:
       Another way is to call `maybeSyncCommittedDataAndOffsets` in finally block of run method before `closeConsumer()`.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] satishd commented on a change in pull request #11058: KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts.

Posted by GitBox <gi...@apache.org>.
satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719295072



##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -120,6 +171,37 @@ public void run() {
         }
     }
 
+    private void maybeSyncCommittedDataAndOffsets(boolean forceSync) {
+        boolean noOffsetUpdates = committedPartitionToConsumedOffsets.equals(partitionToConsumedOffsets);
+        if (noOffsetUpdates || !forceSync && time.milliseconds() - lastSyncedTimeMs < committedOffsetSyncIntervalMs) {
+            log.debug("Skip syncing committed offsets, noOffsetUpdates: {}, forceSync: {}", noOffsetUpdates, forceSync);
+            return;
+        }
+
+        try {
+            // partitionToConsumedOffsets is not getting changed concurrently as this method is called from #run() which updates the same.
+            // Need to take lock on assignPartitionsLock as assignedTopicPartitions might get updated by other threads.
+            synchronized (assignPartitionsLock) {
+                for (TopicIdPartition topicIdPartition : assignedTopicPartitions) {
+                    int metadataPartition = topicPartitioner.metadataPartition(topicIdPartition);
+                    Long offset = partitionToConsumedOffsets.get(metadataPartition);
+                    if (offset != null && !offset.equals(committedPartitionToConsumedOffsets.get(metadataPartition))) {

Review comment:
       This is a known limitation that I want to address in the future. One possible way I was thinking is to keep track of the last processed event’s offset to be maintained in the in-memory store and the last offset written into the snapshot file. If both these entries are the same then we will skip taking a new snapshot and committing it. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org