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


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -383,11 +396,11 @@ public void clear() {
 
     // Visible for testing
     public List<EpochEntry> epochEntries() {
-        lock.writeLock().lock();
+        lock.readLock().lock();

Review Comment:
   Nice!
   
   Could we move this to a separate PR please since I think this is an obvious 
change that can be shipped earlier than this PR.



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -148,6 +151,16 @@ private List<EpochEntry> 
removeWhileMatching(Iterator<Map.Entry<Integer, EpochEn
         return removedEpochs;
     }
 
+    public LeaderEpochFileCache writeTo(LeaderEpochCheckpoint 
leaderEpochCheckpoint) {
+        lock.writeLock().lock();

Review Comment:
   You don't need write locks here since you are flushing to an entirely new 
checkpoint. You need read lock on existing cache. (similar to flush() method)



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

Review Comment:
   It seems like we are doing the following:
   1. Flush the cache values to the checkpoint.
   2. Create a new file cache which would read from the checkpoint created in 
step 1
   
   which means, we are probably trying to have a copy of this cache backed by 
this new checkpoint here.
   
   Alternatively, may I suggest the following:
   
   ```
   public LeaderEpochFileCache 
cloneWithLeaderEpochCheckpoint(LeaderEpochCheckpoint leaderEpochCheckpoint) {
       this.flushToCheckpoint(leaderEpochCheckpoint);
       return new LeaderEpochFileCache(this.topicPartition, 
leaderEpochCheckpoint);
   }
   ```
   `flushToCheckpoint()` is similar to existing flush() method.



##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/InMemoryLeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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
+ */
+public class InMemoryLeaderEpochCheckpoint implements LeaderEpochCheckpoint {

Review Comment:
   Please help me clarify my understanding here. I don't understand the 
motivation here.
   
   File based checkpoints are used for faster recovery so that when a restart 
occurs, we are don't lose the in-memory state and can recover from a 
checkpoint. The concept of in-memory checkpoint contradicts the entire purpose 
of having a checkpoint logic! If we are keeping this in-memory, then why even 
bother checkpointing at all?



##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -219,6 +221,29 @@ class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
     }
   }
 
+  /**
+   * Returns the in memory leader epoch checkpoint by truncating with the 
given start[exclusive] and end[inclusive] offset
+   *
+   * @param log         The actual log from where to take the leader-epoch 
checkpoint
+   * @param startOffset The start offset of the checkpoint file (exclusive in 
the truncation).
+   *                    If start offset is 6, then it will retain an entry at 
offset 6.
+   * @param endOffset   The end offset of the checkpoint file (inclusive in 
the truncation)
+   *                    If end offset is 100, then it will remove the entries 
greater than or equal to 100.
+   * @return the truncated leader epoch checkpoint
+   */
+  private[remote] def getLeaderEpochCheckpoint(log: UnifiedLog, startOffset: 
Long, endOffset: Long): InMemoryLeaderEpochCheckpoint = {

Review Comment:
   where we going to use this? The motivation for this PR is unclear to me.



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