satishd commented on code in PR #13456:
URL: https://github.com/apache/kafka/pull/13456#discussion_r1156800195


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/InMemoryLeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.storage.internals.checkpoint;
+
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class stores a list of EpochEntry(LeaderEpoch + Offsets) to memory
+ *
+ * The motivation for this class is to allow remote log manager to create the 
RemoteLogSegmentMetadata(RLSM)
+ * with the correct leader epoch info for a specific segment. To do that, we 
need to rely on the LeaderEpochCheckpointCache
+ * to truncate from start and end, to get the epoch info. However, we don't 
really want to truncate the epochs in cache
+ * (and write to checkpoint file in the end). So, we introduce this 
InMemoryLeaderEpochCheckpoint to feed into LeaderEpochCheckpointCache,
+ * and when we truncate the epoch for RLSM, we can do them in memory without 
affecting the checkpoint file, and without interacting with file system.
+ */
+public class InMemoryLeaderEpochCheckpoint implements LeaderEpochCheckpoint {
+    private List<EpochEntry> epochs = new ArrayList<>();

Review Comment:
   `epochs` can be initialized as an empty List like `Collections.emptyList()`. 



##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/InMemoryLeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.storage.internals.checkpoint;
+
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class stores a list of EpochEntry(LeaderEpoch + Offsets) to memory
+ *
+ * The motivation for this class is to allow remote log manager to create the 
RemoteLogSegmentMetadata(RLSM)
+ * with the correct leader epoch info for a specific segment. To do that, we 
need to rely on the LeaderEpochCheckpointCache
+ * to truncate from start and end, to get the epoch info. However, we don't 
really want to truncate the epochs in cache
+ * (and write to checkpoint file in the end). So, we introduce this 
InMemoryLeaderEpochCheckpoint to feed into LeaderEpochCheckpointCache,
+ * and when we truncate the epoch for RLSM, we can do them in memory without 
affecting the checkpoint file, and without interacting with file system.
+ */
+public class InMemoryLeaderEpochCheckpoint implements LeaderEpochCheckpoint {
+    private List<EpochEntry> epochs = new ArrayList<>();
+
+    public void write(Collection<EpochEntry> epochs) {
+        this.epochs.addAll(epochs);
+    }
+
+    public List<EpochEntry> read() {
+        return epochs;

Review Comment:
   Return an unmodifiable List like below so that the internal list is 
unmodifiable from out side of this class.
   
   ```
   Collections.unmodifiableList(epochs)
   ```



##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/InMemoryLeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.storage.internals.checkpoint;
+
+import org.apache.kafka.server.common.CheckpointFile;
+import org.apache.kafka.storage.internals.log.EpochEntry;
+
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class stores a list of EpochEntry(LeaderEpoch + Offsets) to memory
+ *
+ * The motivation for this class is to allow remote log manager to create the 
RemoteLogSegmentMetadata(RLSM)
+ * with the correct leader epoch info for a specific segment. To do that, we 
need to rely on the LeaderEpochCheckpointCache
+ * to truncate from start and end, to get the epoch info. However, we don't 
really want to truncate the epochs in cache
+ * (and write to checkpoint file in the end). So, we introduce this 
InMemoryLeaderEpochCheckpoint to feed into LeaderEpochCheckpointCache,
+ * and when we truncate the epoch for RLSM, we can do them in memory without 
affecting the checkpoint file, and without interacting with file system.
+ */
+public class InMemoryLeaderEpochCheckpoint implements LeaderEpochCheckpoint {
+    private List<EpochEntry> epochs = new ArrayList<>();
+
+    public void write(Collection<EpochEntry> epochs) {
+        this.epochs.addAll(epochs);

Review Comment:
   A new ArrayList with the intended collection can be created here and store 
like below. 
   
   ```
   this.epochs = new ArrayList<EpochEntry>(epochs)
   ```



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -147,6 +152,16 @@ private List<EpochEntry> 
removeWhileMatching(Iterator<Map.Entry<Integer, EpochEn
         return removedEpochs;
     }
 
+    public LeaderEpochFileCache 
cloneWithLeaderEpochCheckpoint(LeaderEpochCheckpoint leaderEpochCheckpoint) {
+        lock.readLock().lock();
+        try {
+            flushTo(leaderEpochCheckpoint, epochEntries());

Review Comment:
   `epochs.values()` can be used instead of calling `epochEnrties()` which 
creates a new ArrayList that is not needed 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-unsubscr...@kafka.apache.org

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

Reply via email to