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


##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -25,6 +26,7 @@ import org.apache.kafka.common.record.{RecordBatch, 
RemoteLogInputStream}
 import org.apache.kafka.common.utils.{ChildFirstClassLoader, Utils}
 import 
org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager
 import org.apache.kafka.server.log.remote.storage._
+import 
org.apache.kafka.storage.internals.checkpoint.InMemoryLeaderEpochCheckpoint

Review Comment:
   Is this still required here?



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -383,21 +398,25 @@ public void clear() {
 
     // Visible for testing
     public List<EpochEntry> epochEntries() {
-        lock.writeLock().lock();
+        lock.readLock().lock();
         try {
             return new ArrayList<>(epochs.values());
         } finally {
-            lock.writeLock().unlock();
+            lock.readLock().unlock();
         }
     }
 
-    private void flush() {
+    private void flushTo(LeaderEpochCheckpoint leaderEpochCheckpoint, 
Collection<EpochEntry> epochEntries) {
         lock.readLock().lock();
         try {
-            checkpoint.write(epochs.values());
+            leaderEpochCheckpoint.write(epochEntries);
         } finally {
             lock.readLock().unlock();
         }
     }
 
+    private void flush() {
+        flushTo(this.checkpoint, epochs.values());

Review Comment:
   both invocations for `flushTo()` have the same second parameter.
   
   Could we instead read the entries inside the flushTo method itself? Note 
that we don't require to acquire locks in flushTo() or in 
cloneWithLeaderEpochCheckpoint(), since epochEntries() does it for us.
   
   ```
   private void flushTo(LeaderEpochCheckpoint leaderEpochCheckpoint) {
       leaderEpochCheckpoint.write(epochEntries());
   }
   
   private void flush() {
       flushTo(this.checkpoint);
   }
   
   public LeaderEpochFileCache 
cloneWithLeaderEpochCheckpoint(LeaderEpochCheckpoint leaderEpochCheckpoint) {
       flushTo(leaderEpochCheckpoint);
       // create a new cache backed by the provided leaderEpochCheckpoint
       return new LeaderEpochFileCache(this.topicPartition, 
leaderEpochCheckpoint);
   }
   ```
   



##########
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:
   Thanks for the explanation @showuon. It's clear now. 
   
   To rephrase what you mentioned (correct me if I misunderstood) we want to 
create an in-memory copy of the `LeaderEpochCheckpointCache` so that we can 
modify it without modifying the original checkpointcache & checkpoint. Is that 
right?
   
   At the end of the day, RLSM requires `Map<Int, Long> segmentLeaderEpochs()` 
and the source is `LeaderEpochCheckpointCache`. I was wondering if it would be 
possible to copy the `LeaderEpochCheckpointCache` into an intermediate data 
structure (which doesn't have to be of type LeaderEpochCheckpointCache, hence 
decoupling it from the need to create a dummy `InMemoryLeaderEpochCheckpoint `) 
and then using that intermediate data structure to extract the required Map 
(after whatever manipulation we want to do with it).
   
   Similar logic could be employed for the requirement to provide `ByteBuffer 
leaderEpochIndex` to `LogSegmentData`.
   
   Thoughts?
   



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