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 2022/12/24 01:48:47 UTC

[GitHub] [kafka] satishd opened a new pull request, #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

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

   KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.
   
   ### 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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1063708632


##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -65,9 +65,14 @@ class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureCh
   val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
     OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent)
 
-  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets)
+  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets.map{case (k, v)=> Tuple2(k, v)}.toSeq.asJava)

Review Comment:
   The `map` operation is a `no-op` here since we are converting a tuple to a tuple,



-- 
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] ijuma commented on pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13046:
URL: https://github.com/apache/kafka/pull/13046#issuecomment-1374023466

   I have a few more comments on this PR, will get back to it soon.


-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097364723


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   The reason for using write lock is not to allow multiple threads to write concurrently and no updates on epochs. But we can have a constraint on `LeaderEpochCheckpoint#write` implementation to be thread-safe(the implementation already follows this constraint) and use a read lock here. So, as you suggested, taking a read lock here should be sufficient with the mentioned constraint. 



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1070638073


##########
server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java:
##########
@@ -72,7 +71,7 @@ public CheckpointFile(File file,
         tempPath = Paths.get(absolutePath.toString() + ".tmp");
     }
 
-    public void write(Collection<T> entries) throws IOException {
+    public void write(List<T> entries) throws IOException {

Review Comment:
   I don't think consistency is necessarily a goal in cases like this. The original intent was to accept the minimal interface required, but to return something more specific to avoid unnecessary conversions in both sides.



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097245192


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);

Review Comment:
   `assign(EpochEntry entry)` is already a private method that does not pass this instance outside this class. Am I 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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097325246


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);

Review Comment:
   My bad, we are calling the private `assign` overload., not the public one.



-- 
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 diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1059247572


##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -62,12 +62,17 @@ trait OffsetCheckpoint {
  *  -----checkpoint file end----------
  */
 class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureChannel = null) {
-  val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
+  val checkpoint = new CheckpointFileWithFailureHandler[Tuple2[TopicPartition, Long]](file, OffsetCheckpointFile.CurrentVersion,

Review Comment:
   Nice catch! This is not needed, it seemed to be part of earlier refactoring.



-- 
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 diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
junrao commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1063005106


##########
core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala:
##########
@@ -243,282 +245,287 @@ class LeaderEpochFileCacheTest {
 
     //Given
     val cache = new LeaderEpochFileCache(tp, checkpoint)
-    cache.assign(epoch = 2, startOffset = 6)
+    cache.assign(2, 6)
 
     //When
     val checkpoint2 = new LeaderEpochCheckpointFile(new File(checkpointPath))
     val cache2 = new LeaderEpochFileCache(tp, checkpoint2)
 
     //Then
     assertEquals(1, cache2.epochEntries.size)
-    assertEquals(EpochEntry(2, 6), cache2.epochEntries.toList(0))
+    assertEquals(new EpochEntry(2, 6), cache2.epochEntries.get(0))
   }
 
   @Test
   def shouldEnforceMonotonicallyIncreasingEpochs(): Unit = {
     //Given
-    cache.assign(epoch = 1, startOffset = 5);
+    cache.assign(1, 5);
     var logEndOffset = 6
-    cache.assign(epoch = 2, startOffset = 6);
+    cache.assign(2, 6);
     logEndOffset = 7
 
     //When we update an epoch in the past with a different offset, the log has already reached
     //an inconsistent state. Our options are either to raise an error, ignore the new append,
     //or truncate the cached epochs to the point of conflict. We take this latter approach in
     //order to guarantee that epochs and offsets in the cache increase monotonically, which makes
     //the search logic simpler to reason about.
-    cache.assign(epoch = 1, startOffset = 7);
+    cache.assign(1, 7);
     logEndOffset = 8
 
     //Then later epochs will be removed
-    assertEquals(Some(1), cache.latestEpoch)
+    assertEquals(Optional.of(1), cache.latestEpoch)
 
     //Then end offset for epoch 1 will have changed
-    assertEquals((1, 8), cache.endOffsetFor(1, logEndOffset))
+    assertEquals((1, 8), toTuple(cache.endOffsetFor(1, logEndOffset)))
 
     //Then end offset for epoch 2 is now undefined
-    assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), cache.endOffsetFor(2, logEndOffset))
-    assertEquals(EpochEntry(1, 7), cache.epochEntries(0))
+    assertEquals((UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET), toTuple(cache.endOffsetFor(2, logEndOffset)))
+    assertEquals(new EpochEntry(1, 7), cache.epochEntries.get(0))
+  }
+
+  def toTuple[K, V](entry: java.util.Map.Entry[K, V]): (K, V) = {

Review Comment:
   Could this be private?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1294,36 +1293,53 @@ class UnifiedLog(@volatile var logStartOffset: Long,
         // The first cached epoch usually corresponds to the log start offset, but we have to verify this since
         // it may not be true following a message format version bump as the epoch will not be available for
         // log entries written in the older format.
-        val earliestEpochEntry = leaderEpochCache.flatMap(_.earliestEntry)
-        val epochOpt = earliestEpochEntry match {
-          case Some(entry) if entry.startOffset <= logStartOffset => Optional.of[Integer](entry.epoch)
-          case _ => Optional.empty[Integer]()
+        val earliestEpochEntry = leaderEpochCache match {
+          case Some(cache) => cache.earliestEntry()
+          case None => Optional.empty[EpochEntry]()
         }
+
+        val epochOpt = if (earliestEpochEntry.isPresent && earliestEpochEntry.get().startOffset <= logStartOffset) {
+          Optional.of[Integer](earliestEpochEntry.get().epoch)
+        } else Optional.empty[Integer]()
+
         Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logStartOffset, epochOpt))
       } else if (targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) {
         val curLocalLogStartOffset = localLogStartOffset()
-        val earliestLocalLogEpochEntry = leaderEpochCache.flatMap(cache =>
-          cache.epochForOffset(curLocalLogStartOffset).flatMap(cache.epochEntry))
-        val epochOpt = earliestLocalLogEpochEntry match {
-          case Some(entry) if entry.startOffset <= curLocalLogStartOffset => Optional.of[Integer](entry.epoch)
-          case _ => Optional.empty[Integer]()
+
+        val earliestLocalLogEpochEntry: Optional[EpochEntry] = leaderEpochCache match {
+          case Some(cache) =>
+            val value = cache.epochForOffset(curLocalLogStartOffset)
+            if (value.isPresent) cache.epochEntry(value.get) else Optional.empty[EpochEntry]()
+          case None => Optional.empty[EpochEntry]()
         }
+
+        val epochOpt = if (earliestLocalLogEpochEntry.isPresent && earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset)
+          Optional.of[Integer](earliestLocalLogEpochEntry.get().epoch)
+        else Optional.empty[Integer]()
+
         Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochOpt))
       } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) {
-        val latestEpochOpt = leaderEpochCache.flatMap(_.latestEpoch).map(_.asInstanceOf[Integer])
-        val epochOptional = Optional.ofNullable(latestEpochOpt.orNull)
-        Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, epochOptional))
+

Review Comment:
   extra newline



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   Yes, the proposed package names sound good 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] Hangleton commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1094474509


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.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.storage.internals.checkpoint;
+
+import org.apache.kafka.server.common.CheckpointFile.EntryFormatter;
+import org.apache.kafka.server.log.internals.CheckpointFileWithFailureHandler;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.server.log.internals.LogDirFailureChannel;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class persists a map of (LeaderEpoch => Offsets) to a file (for a certain replica)
+ * <p>
+ * The format in the LeaderEpoch checkpoint file is like this:
+ * -----checkpoint file begin------
+ * 0                <- LeaderEpochCheckpointFile.currentVersion
+ * 2                <- following entries size
+ * 0  1     <- the format is: leader_epoch(int32) start_offset(int64)
+ * 1  2
+ * -----checkpoint file end----------
+ */
+public class LeaderEpochCheckpointFile implements LeaderEpochCheckpoint {
+
+    public static final Formatter FORMATTER = new Formatter();
+
+    private static final String LEADER_EPOCH_CHECKPOINT_FILENAME = "leader-epoch-checkpoint";
+    private static final Pattern WHITE_SPACES_PATTERN = Pattern.compile("\\s+");
+    private static final int CURRENT_VERSION = 0;
+
+    private final CheckpointFileWithFailureHandler<EpochEntry> checkpoint;
+
+    public LeaderEpochCheckpointFile(File file, LogDirFailureChannel logDirFailureChannel) throws IOException {
+        checkpoint = new CheckpointFileWithFailureHandler<>(file, CURRENT_VERSION, FORMATTER, logDirFailureChannel, file.getParentFile().getParent());
+    }
+
+    public LeaderEpochCheckpointFile(File file) throws IOException {
+        this(file, null);

Review Comment:
   Do we really want to expose this constructor since nothing prevents an NPE down the line. Should `LogDirFailureChannel` be mocked in unit tests instead?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().get() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+            log.trace("Processed end offset request for epoch {} and returning epoch {} " + "with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after " + "truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            EpochEntry firstBeforeStartOffset;
+            if (!removedEntries.isEmpty()) {
+                firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            OptionalInt previousEpoch = OptionalInt.empty();
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+                if (startOffset == offset) return OptionalInt.of(epoch);
+                if (startOffset > offset) return previousEpoch;
+
+                previousEpoch = OptionalInt.of(epoch);

Review Comment:
   We could use a primitive here to avoid the cost of creating an optional until the value to return is found.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().get() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+            log.trace("Processed end offset request for epoch {} and returning epoch {} " + "with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after " + "truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            EpochEntry firstBeforeStartOffset;
+            if (!removedEntries.isEmpty()) {
+                firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            OptionalInt previousEpoch = OptionalInt.empty();
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+                if (startOffset == offset) return OptionalInt.of(epoch);
+                if (startOffset > offset) return previousEpoch;
+
+                previousEpoch = OptionalInt.of(epoch);
+            }
+
+            return previousEpoch;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        return new ArrayList<>(epochs.values());
+    }
+
+    public void flush() {

Review Comment:
   Ditto.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/CheckpointFileWithFailureHandler.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+public class CheckpointFileWithFailureHandler<T> {
+
+    public final File file;
+    private final LogDirFailureChannel logDirFailureChannel;
+    private final String logDir;
+
+    private final CheckpointFile<T> checkpointFile;
+
+    public CheckpointFileWithFailureHandler(File file, int version, CheckpointFile.EntryFormatter<T> formatter,
+                                            LogDirFailureChannel logDirFailureChannel, String logDir) throws IOException {
+        this.file = file;
+        this.logDirFailureChannel = logDirFailureChannel;
+        this.logDir = logDir;
+        checkpointFile = new CheckpointFile<>(file, version, formatter);

Review Comment:
   Curious - the disk failure handler plays an important part in handling I/O failures. What is the reason for not enforcing the same I/O failure path for all types of checkpoint files?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().get() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+            log.trace("Processed end offset request for epoch {} and returning epoch {} " + "with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after " + "truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            EpochEntry firstBeforeStartOffset;
+            if (!removedEntries.isEmpty()) {
+                firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            OptionalInt previousEpoch = OptionalInt.empty();
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+                if (startOffset == offset) return OptionalInt.of(epoch);
+                if (startOffset > offset) return previousEpoch;
+
+                previousEpoch = OptionalInt.of(epoch);
+            }
+
+            return previousEpoch;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {

Review Comment:
   Public but not protected by a lock.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();

Review Comment:
   Flush only if at least one epoch was assigned?



-- 
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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13046:
URL: https://github.com/apache/kafka/pull/13046#issuecomment-1417921173

   Thanks @ijuma for your comments. Addressed them with inline replies and/or with the latest commits.


-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1056745406


##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -62,12 +62,17 @@ trait OffsetCheckpoint {
  *  -----checkpoint file end----------
  */
 class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureChannel = null) {
-  val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
+  val checkpoint = new CheckpointFileWithFailureHandler[Tuple2[TopicPartition, Long]](file, OffsetCheckpointFile.CurrentVersion,

Review Comment:
   What is the reason for this change?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/EpochEntry.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internals;
+
+import java.util.Objects;
+
+// Mapping of epoch to the first offset of the subsequent epoch
+public class EpochEntry {
+    public final int epoch;
+    public final long startOffset;
+
+    public EpochEntry(int epoch, long startOffset) {
+        this.epoch = epoch;
+        this.startOffset = startOffset;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        EpochEntry that = (EpochEntry) o;
+        return epoch == that.epoch && startOffset == that.startOffset;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(epoch, startOffset);

Review Comment:
   This results in boxing of the two elements and the allocation of an array. To avoid unexpected regressions, can we please stick with an allocation free implementation?



##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -255,7 +256,7 @@ class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
     }
 
     // Get the respective epoch in which the starting-offset exists.
-    var maybeEpoch = leaderEpochCache.epochForOffset(startingOffset)
+    var maybeEpoch = leaderEpochCache.epochForOffset(startingOffset).asScala

Review Comment:
   Similarly here, can we just use `Optional` instead of converting to `Option`?



##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -2650,7 +2647,7 @@ class PartitionTest extends AbstractPartitionTest {
     assertEquals(Some(0L), partition.leaderEpochStartOffsetOpt)
 
     val leaderLog = partition.localLogOrException
-    assertEquals(Some(EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.flatMap(_.latestEntry))
+    assertEquals(Some(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.flatMap(_.latestEntry.asScala))

Review Comment:
   We can change the expected value to be `Optional.of` instead of converting the response with `asScala`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/EpochEntry.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internals;
+
+import java.util.Objects;
+
+// Mapping of epoch to the first offset of the subsequent epoch
+public class EpochEntry {
+    public final int epoch;
+    public final long startOffset;
+
+    public EpochEntry(int epoch, long startOffset) {
+        this.epoch = epoch;
+        this.startOffset = startOffset;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        EpochEntry that = (EpochEntry) o;
+        return epoch == that.epoch && startOffset == that.startOffset;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(epoch, startOffset);
+    }
+
+    @Override
+    public String toString() {
+        return "EpochEntry{" +
+                "epoch=" + epoch +
+                ", startOffset=" + startOffset +
+                '}';

Review Comment:
   We typically use `(` and `)` versus `{` and `}` in our `toString` implementations.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpointFile.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.server.common.CheckpointFile.EntryFormatter;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class persists a map of (LeaderEpoch => Offsets) to a file (for a certain replica)
+ * <p>
+ * The format in the LeaderEpoch checkpoint file is like this:
+ * -----checkpoint file begin------
+ * 0                <- LeaderEpochCheckpointFile.currentVersion
+ * 2                <- following entries size
+ * 0  1     <- the format is: leader_epoch(int32) start_offset(int64)
+ * 1  2
+ * -----checkpoint file end----------
+ */
+public class LeaderEpochCheckpointFile implements LeaderEpochCheckpoint {
+    private static final String LEADER_EPOCH_CHECKPOINT_FILENAME = "leader-epoch-checkpoint";
+    private static final Pattern WHITE_SPACES_PATTERN = Pattern.compile("\\s+");
+    private static final int CURRENT_VERSION = 0;
+
+    public static final Formatter FORMATTER = new Formatter();

Review Comment:
   Please have the public field(s) before the private ones.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -995,11 +994,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
-  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch)
+  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch.asScala).map(Int.unbox(_))

Review Comment:
   Can we use `OptionalInt` here?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1286,31 +1286,31 @@ class UnifiedLog(@volatile var logStartOffset: Long,
         // The first cached epoch usually corresponds to the log start offset, but we have to verify this since
         // it may not be true following a message format version bump as the epoch will not be available for
         // log entries written in the older format.
-        val earliestEpochEntry = leaderEpochCache.flatMap(_.earliestEntry)
+        val earliestEpochEntry = leaderEpochCache.flatMap(_.earliestEntry().asScala)

Review Comment:
   To avoid multiple conversions, I think we can simply use Optional here and in the lines below?



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097338495


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   Thinking about it, how about we switch the write lock to a read lock and leave the rest as it is?



-- 
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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13046:
URL: https://github.com/apache/kafka/pull/13046#issuecomment-1417928249

   Thanks @junrao for your review, replied to your comment [inline](https://github.com/apache/kafka/pull/13046#discussion_r1096654123).


-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097238384


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   We should have a lock here as this method is accessing epoch entries which can be added/removed from different threads.



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   We should have a lock here as this method is accessing epoch entries which can be added/removed from other threads.



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1071702185


##########
server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java:
##########
@@ -72,7 +71,7 @@ public CheckpointFile(File file,
         tempPath = Paths.get(absolutePath.toString() + ".tmp");
     }
 
-    public void write(Collection<T> entries) throws IOException {
+    public void write(List<T> entries) throws IOException {

Review Comment:
   Sure, I am fine with the suggestion for now. I updated PR with the changes to keep it like earlier. But this kind of contract looks fragile, we can take a relook at whether we can avoid any conversions and keep the APIs consistent. Filed https://issues.apache.org/jira/browse/KAFKA-14625 to followup later.



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1056748447


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   That said, should this be in the `storage` module at all?



-- 
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 diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1059039639


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -995,11 +994,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
-  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch)
+  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch.asScala).map(Int.unbox(_))

Review Comment:
   Avoided using Java Optional classes here as all its usages are in Scala classes for now. I prefer making this kind of change as part of moving UnifiedLog to storage module. wdyt? 
   



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   +1 on that. I prefer have the package name with `server` but I am fine to omit that to be consistent with the existing convention with other modules. 
   I will address it in the next commit of this PR. Let me know if you are fine with the package name as `org.apache.kafka.storage.internals.checkpoint`. 



##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -2650,7 +2647,7 @@ class PartitionTest extends AbstractPartitionTest {
     assertEquals(Some(0L), partition.leaderEpochStartOffsetOpt)
 
     val leaderLog = partition.localLogOrException
-    assertEquals(Some(EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.flatMap(_.latestEntry))
+    assertEquals(Some(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.flatMap(_.latestEntry.asScala))

Review Comment:
   `flatMap` has to return `Option`, that is why I converted from `Optional` to `Option`. Another way can be to use match case statements that makes it verbose. I left with Scala Option conversions for now as these will be converted to Java later.



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1096704547


##########
storage/src/main/java/org/apache/kafka/server/log/internals/CheckpointFileWithFailureHandler.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+public class CheckpointFileWithFailureHandler<T> {
+
+    public final File file;
+    private final LogDirFailureChannel logDirFailureChannel;
+    private final String logDir;
+
+    private final CheckpointFile<T> checkpointFile;
+
+    public CheckpointFileWithFailureHandler(File file, int version, CheckpointFile.EntryFormatter<T> formatter,
+                                            LogDirFailureChannel logDirFailureChannel, String logDir) throws IOException {
+        this.file = file;
+        this.logDirFailureChannel = logDirFailureChannel;
+        this.logDir = logDir;
+        checkpointFile = new CheckpointFile<>(file, version, formatter);

Review Comment:
   Each checkpoint file does not need to be treated the same. Whichever checkpoint file needs to treat these errors as disk failure will use the handler. 



-- 
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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13046:
URL: https://github.com/apache/kafka/pull/13046#issuecomment-1420265571

    These test failures seem to be unrelated. I rebased the PR against trunk. I will wait for the test results.  


-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1099624434


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   Filed [KAFKA-14688](https://issues.apache.org/jira/browse/KAFKA-14688) and it is addressed with https://github.com/apache/kafka/pull/13213



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1080718765


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {

Review Comment:
   Should this and `previousEpoch` be `OptionalInt`?



##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -65,9 +65,14 @@ class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureCh
   val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
     OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent)
 
-  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets)
+  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets.toSeq.asJava)
 
-  def read(): Map[TopicPartition, Long] = checkpoint.read().toMap
+  def read(): Map[TopicPartition, Long] = {
+    val list = checkpoint.read()
+    val result = mutable.Map.empty[TopicPartition, Long]
+    list.forEach { case (tp, offset) => result += tp -> offset }
+    result.toMap

Review Comment:
   Can we avoid the conversion to an immutable map here? The additional copy doesn't buy us anything.



##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.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.storage.internals.checkpoint;
+
+import org.apache.kafka.server.common.CheckpointFile.EntryFormatter;
+import org.apache.kafka.server.log.internals.CheckpointFileWithFailureHandler;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.server.log.internals.LogDirFailureChannel;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class persists a map of (LeaderEpoch => Offsets) to a file (for a certain replica)
+ * <p>
+ * The format in the LeaderEpoch checkpoint file is like this:
+ * -----checkpoint file begin------
+ * 0                <- LeaderEpochCheckpointFile.currentVersion
+ * 2                <- following entries size
+ * 0  1     <- the format is: leader_epoch(int32) start_offset(int64)
+ * 1  2
+ * -----checkpoint file end----------
+ */
+public class LeaderEpochCheckpointFile implements LeaderEpochCheckpoint {
+
+    public static final Formatter FORMATTER = new Formatter();
+
+    private static final String LEADER_EPOCH_CHECKPOINT_FILENAME = "leader-epoch-checkpoint";
+    private static final Pattern WHITE_SPACES_PATTERN = Pattern.compile("\\s+");
+    private static final int CURRENT_VERSION = 0;
+
+    private final CheckpointFileWithFailureHandler<EpochEntry> checkpoint;
+
+    public LeaderEpochCheckpointFile(File file, LogDirFailureChannel logDirFailureChannel) throws IOException {
+        checkpoint = new CheckpointFileWithFailureHandler<>(file, CURRENT_VERSION, FORMATTER, logDirFailureChannel, file.getParentFile().getParent());
+    }
+
+    public LeaderEpochCheckpointFile(File file) throws IOException {
+        this(file, null);
+    }
+
+    public void write(Collection<EpochEntry> epochs) {
+        checkpoint.write(epochs);
+    }
+
+    public List<EpochEntry> read() {
+        return checkpoint.read();
+    }
+
+    public static File newFile(File dir) {
+        return new File(dir, LEADER_EPOCH_CHECKPOINT_FILENAME);
+    }
+
+    public static class Formatter implements EntryFormatter<EpochEntry> {

Review Comment:
   Does this need to be public?



##########
core/src/main/scala/kafka/log/LogSegment.scala:
##########
@@ -249,9 +248,10 @@ class LogSegment private[log] (val log: FileRecords,
     if (batch.hasProducerId) {
       val producerId = batch.producerId
       val appendInfo = producerStateManager.prepareUpdate(producerId, origin = AppendOrigin.REPLICATION)
-      val maybeCompletedTxn = appendInfo.append(batch, Optional.empty()).asScala
+      val maybeCompletedTxn = appendInfo.append(batch, Optional.empty())
       producerStateManager.update(appendInfo)
-      maybeCompletedTxn.foreach { completedTxn =>
+      if (maybeCompletedTxn.isPresent) {

Review Comment:
   Nit: `ifPresent` is the equivalent to `foreach`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {

Review Comment:
   This was private before, why is it now public?



##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -65,9 +65,14 @@ class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureCh
   val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
     OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent)
 
-  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets)
+  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets.toSeq.asJava)

Review Comment:
   Is this `toSeq` call still required?



##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -65,9 +65,14 @@ class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureCh
   val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
     OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent)
 
-  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets)
+  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets.toSeq.asJava)
 
-  def read(): Map[TopicPartition, Long] = checkpoint.read().toMap
+  def read(): Map[TopicPartition, Long] = {
+    val list = checkpoint.read()
+    val result = mutable.Map.empty[TopicPartition, Long]

Review Comment:
   We should size the map appropriately.



##########
core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala:
##########
@@ -65,9 +65,14 @@ class OffsetCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureCh
   val checkpoint = new CheckpointFileWithFailureHandler[(TopicPartition, Long)](file, OffsetCheckpointFile.CurrentVersion,
     OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent)
 
-  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets)
+  def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets.toSeq.asJava)
 
-  def read(): Map[TopicPartition, Long] = checkpoint.read().toMap
+  def read(): Map[TopicPartition, Long] = {
+    val list = checkpoint.read()
+    val result = mutable.Map.empty[TopicPartition, Long]
+    list.forEach { case (tp, offset) => result += tp -> offset }

Review Comment:
   You can avoid the tuple creation by using something like:
   
   `result(tp) = offset`



##########
storage/src/main/java/org/apache/kafka/server/log/internals/CheckpointFileWithFailureHandler.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.internals;

Review Comment:
   Why is this not in the checkpoint package?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/EpochEntry.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internals;

Review Comment:
   This was previously under the `epoch` package. Should we do the same, i.e. `storage.internals.epoch`? Same for `LeaderEpochFileCache`.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);

Review Comment:
   The original code did this under a write lock. Did we establish that it wasn't required?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();

Review Comment:
   Nit: can this be inlined?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().get() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+            log.trace("Processed end offset request for epoch {} and returning epoch {} " + "with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after " + "truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());

Review Comment:
   No need for string concat here.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().get() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+            log.trace("Processed end offset request for epoch {} and returning epoch {} " + "with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after " + "truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            EpochEntry firstBeforeStartOffset;

Review Comment:
   Why do we need this before the `if`?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().get() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+            log.trace("Processed end offset request for epoch {} and returning epoch {} " + "with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());

Review Comment:
   No need for the string concat here. Also, since this is a trace log, we may want to have it under a if check (to avoid the array allocation for the parameters, etc.).



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1056748412


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   Looks like this was in the checkpoints package previously. Would it make sense to keep that distinction here? It could be `org.apache.kafka.server.checkpoint.internals` or something along those lines.



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1094564674


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.log.internals.EpochEntry;
+
+import java.util.Collection;
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {
+
+    void write(Collection<EpochEntry> epochs);

Review Comment:
   @junrao The previous code didn't do that and if we make that change, we have to make additional collection copies in a few places. @satishd filed a separate JIRA to look into this separately (and decide whether it's worth 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 a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1096705076


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);

Review Comment:
   Taking write lock of the same instance while initializing the instance does not make sense as the instance is not yet available for access by any other thread while its constructor is still in progress. 



-- 
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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

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

   Thanks @ijuma for the updated comments. Addressed them with inline and update the PR with 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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13046:
URL: https://github.com/apache/kafka/pull/13046#issuecomment-1420484181

   Test failures do not seem to be related to the PR changes, merging to trunk. 


-- 
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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

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

   @ijuma I rebased with trunk and resolved conflicts. Please review it when you get some time.


-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1093672007


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.log.internals.EpochEntry;
+
+import java.util.Collection;
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {
+
+    void write(Collection<EpochEntry> epochs);

Review Comment:
   Should we change the input of write to a List to indicate that ordering is important and to be more consistent with the return value of read()?



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097238384


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   We should have a lock here as this method is accessing epoch entries which can be added/removed through other methods from different threads.



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097332224


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   Yes, but isn't `flush` only called with the lock held? Looking at the usage more closely, this is the case for every usage except `assign`. So, it does look like we have a thread safety bug for the `assign` case. However, the change here has a couple of issues:
   1. Holds the lock for the duration of the flush call blocking read operations, which is unnecessary.
   2. Introduces new re-entrant lock calls that were not required previously.



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1096654123


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.log.internals.EpochEntry;
+
+import java.util.Collection;
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {
+
+    void write(Collection<EpochEntry> epochs);

Review Comment:
   @junrao This was discussed in an earlier comment [here](https://github.com/apache/kafka/pull/13046#discussion_r1066636586).



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1094564674


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.log.internals.EpochEntry;
+
+import java.util.Collection;
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {
+
+    void write(Collection<EpochEntry> epochs);

Review Comment:
   @junrao The previous code didn't do that and if we make that change, we have to make additional collection copies in a few places. @satishd filed a separate JIRA to look into this separately.



-- 
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 merged pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd merged PR #13046:
URL: https://github.com/apache/kafka/pull/13046


-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097335270


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   Also, it's not clear why a `write` lock is required for this operation since we are only reading from `epochs` (the actual checkpoint write has a separate lock).



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097332224


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   Yes, but isn't `flush` only called with the lock held? Looking at the usage more closely, this is the case for every usage except `assign`. So, it does look like we have a thread safety bug for the `assign` case. However, the change here has a couple of issues:



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1056750161


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   It makes sense to me that it should be. But perhaps the package structure for this module should be `org.apache.kafka.server.storage.internals.*` versus the one we've been using for the log layer. Maybe we can use this approach for the checkpoint code and move the log layer all at once at the end.



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1066641727


##########
storage/src/main/java/org/apache/kafka/server/log/internals/CheckpointFileWithFailureHandler.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.server.common.CheckpointFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+public class CheckpointFileWithFailureHandler<T> {
+
+    public final File file;
+    public final int version;

Review Comment:
   `version` was private in the `Scala` implementation, why is it public here?



##########
storage/src/main/java/org/apache/kafka/server/log/internals/EpochEntry.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internals;
+
+// Mapping of epoch to the first offset of the subsequent epoch
+public class EpochEntry {
+    public final int epoch;
+    public final long startOffset;
+
+    public EpochEntry(int epoch, long startOffset) {
+        this.epoch = epoch;
+        this.startOffset = startOffset;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        EpochEntry that = (EpochEntry) o;
+        return epoch == that.epoch && startOffset == that.startOffset;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = epoch;
+        result = 31 * result + (int) (startOffset ^ (startOffset >>> 32));

Review Comment:
   Use `Long.hashCode`.



##########
server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java:
##########
@@ -72,7 +71,7 @@ public CheckpointFile(File file,
         tempPath = Paths.get(absolutePath.toString() + ".tmp");
     }
 
-    public void write(Collection<T> entries) throws IOException {
+    public void write(List<T> entries) throws IOException {

Review Comment:
   Why did we change this?



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1063709428


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        Optional<EpochEntry> lastEntry = latestEntry();
+        return lastEntry.map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    public void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public Optional<Integer> latestEpoch() {
+        return latestEntry().map(x -> x.epoch);
+    }
+
+    public Optional<Integer> previousEpoch() {
+        lock.readLock().lock();
+        try {
+            return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))).map(Map.Entry::getKey);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<Integer> previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<Integer> nextEpoch(int epoch) {

Review Comment:
   Can we use `OptionalInt` 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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1062119225


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   I originally also preferred having `server` in the package name, but on further thought, there are advantages in basically having the module name as the package name after `kafka`. It avoids accidental split packages across modules, which can cause problems if we ever decide to use Java 9 modules.
   
   If we agree on that approach, then the module names would be `org.apache.kafka.storage.internals.checkpoint` and `org.apache.kafka.storage.internals.log` (one annoying aspect of using `log` is that GitHub filters it from their code navigation, but we probably have to live with that for now).
   
   @junrao what are your 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] satishd commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097364723


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   The reason for using write lock is not to allow multiple threads to write concurrently. But we can have a constraint on `LeaderEpochCheckpoint#write` implementation to be thread safe and use a read lock here. So, as you suggested, taking a read lock here should be sufficient with the mentioned constraint. 



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097364723


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   The reason for using write lock is not to allow multiple threads to write concurrently. But we can have a constraint on `LeaderEpochCheckpoint#write` implementation to be thread-safe(the implementation already follows this constraint) and use a read lock here. So, as you suggested, taking a read lock here should be sufficient with the mentioned constraint. 



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1096797014


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;

Review Comment:
   Replacing `OptionalInt` with `Integer` (I believe this was a recent update) doesn't make things better. Both of them box, right? I think the code you had previously was better.



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();

Review Comment:
   This should be `!isEmpty` as some list implementations have a cheaper `isEmpty` than `size()`.



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();

Review Comment:
   We added a lock that wasn't there before, but this one seems ok since it's only used by tests.



##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);

Review Comment:
   We are calling a public non-final method from here, so we can publish the instance in an unsafe manner. To make  this change safe, we should make the `assign` method final so it's clear that the instance doesn't escape the constructor.



##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   We added a lock here that wasn't there before. It looks like this is only used within this class. Can we make it private and remove the lock?



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1063705498


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -995,11 +994,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
-  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch)
+  def latestEpoch: Option[Int] = leaderEpochCache.flatMap(_.latestEpoch.asScala).map(Int.unbox(_))

Review Comment:
   Yes, if all usages are in Scala, it makes sense to keep it 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 diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1063953030


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   I moved checkpoint files to `org.apache.kafka.storage.internals.checkpoint` with the latest commits as mentioned earlier. We can have a separate followup PR for moving existing `log.internals`.



-- 
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] ijuma commented on pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13046:
URL: https://github.com/apache/kafka/pull/13046#issuecomment-1374714539

   @satishd When you have some cycles, please resolve the conflicts on this PR. I think we can focus on merging this next and then #13040.


-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1056748020


##########
storage/src/main/java/org/apache/kafka/server/log/internals/EpochEntry.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.internals;
+
+import java.util.Objects;
+
+// Mapping of epoch to the first offset of the subsequent epoch
+public class EpochEntry {
+    public final int epoch;
+    public final long startOffset;
+
+    public EpochEntry(int epoch, long startOffset) {
+        this.epoch = epoch;
+        this.startOffset = startOffset;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        EpochEntry that = (EpochEntry) o;
+        return epoch == that.epoch && startOffset == that.startOffset;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(epoch, startOffset);

Review Comment:
   This results in boxing of the two elements and the allocation of an array. To avoid unexpected performance regressions, can we please stick with an allocation free implementation?



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1056752254


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   Thinking about this some more, to remain consistent with other packages like `raft`, `metadata` and others, we could skip the `server` and just have `org.apache.kafka.storage.internals.*` for the non api classes.



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1096705686


##########
storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.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.storage.internals.checkpoint;
+
+import org.apache.kafka.server.common.CheckpointFile.EntryFormatter;
+import org.apache.kafka.server.log.internals.CheckpointFileWithFailureHandler;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.server.log.internals.LogDirFailureChannel;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.regex.Pattern;
+
+/**
+ * This class persists a map of (LeaderEpoch => Offsets) to a file (for a certain replica)
+ * <p>
+ * The format in the LeaderEpoch checkpoint file is like this:
+ * -----checkpoint file begin------
+ * 0                <- LeaderEpochCheckpointFile.currentVersion
+ * 2                <- following entries size
+ * 0  1     <- the format is: leader_epoch(int32) start_offset(int64)
+ * 1  2
+ * -----checkpoint file end----------
+ */
+public class LeaderEpochCheckpointFile implements LeaderEpochCheckpoint {
+
+    public static final Formatter FORMATTER = new Formatter();
+
+    private static final String LEADER_EPOCH_CHECKPOINT_FILENAME = "leader-epoch-checkpoint";
+    private static final Pattern WHITE_SPACES_PATTERN = Pattern.compile("\\s+");
+    private static final int CURRENT_VERSION = 0;
+
+    private final CheckpointFileWithFailureHandler<EpochEntry> checkpoint;
+
+    public LeaderEpochCheckpointFile(File file, LogDirFailureChannel logDirFailureChannel) throws IOException {
+        checkpoint = new CheckpointFileWithFailureHandler<>(file, CURRENT_VERSION, FORMATTER, logDirFailureChannel, file.getParentFile().getParent());
+    }
+
+    public LeaderEpochCheckpointFile(File file) throws IOException {
+        this(file, null);

Review Comment:
   Good catch! 



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1097469398


##########
storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java:
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.epoch;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.log.internals.EpochEntry;
+import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Predicate;
+
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
+import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
+
+/**
+ * Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
+ * <p>
+ * Leader Epoch = epoch assigned to each leader by the controller.
+ * Offset = offset of the first message in each epoch.
+ */
+public class LeaderEpochFileCache {
+    private final LeaderEpochCheckpoint checkpoint;
+    private final Logger log;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final TreeMap<Integer, EpochEntry> epochs = new TreeMap<>();
+
+    /**
+     * @param topicPartition the associated topic partition
+     * @param checkpoint     the checkpoint file
+     */
+    public LeaderEpochFileCache(TopicPartition topicPartition, LeaderEpochCheckpoint checkpoint) {
+        this.checkpoint = checkpoint;
+        LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] ");
+        log = logContext.logger(LeaderEpochFileCache.class);
+        checkpoint.read().forEach(this::assign);
+    }
+
+    /**
+     * Assigns the supplied Leader Epoch to the supplied Offset
+     * Once the epoch is assigned it cannot be reassigned
+     */
+    public void assign(int epoch, long startOffset) {
+        EpochEntry entry = new EpochEntry(epoch, startOffset);
+        if (assign(entry)) {
+            log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            flush();
+        }
+    }
+
+    public void assign(List<EpochEntry> entries) {
+        entries.forEach(entry -> {
+            if (assign(entry)) {
+                log.debug("Appended new epoch entry {}. Cache now contains {} entries.", entry, epochs.size());
+            }
+        });
+        if (entries.size() > 0) flush();
+    }
+
+    private boolean isUpdateNeeded(EpochEntry entry) {
+        return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true);
+    }
+
+    private boolean assign(EpochEntry entry) {
+        if (entry.epoch < 0 || entry.startOffset < 0) {
+            throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry);
+        }
+
+        // Check whether the append is needed before acquiring the write lock
+        // in order to avoid contention with readers in the common case
+        if (!isUpdateNeeded(entry)) return false;
+
+        lock.writeLock().lock();
+        try {
+            if (isUpdateNeeded(entry)) {
+                maybeTruncateNonMonotonicEntries(entry);
+                epochs.put(entry.epoch, entry);
+                return true;
+            } else {
+                return false;
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Remove any entries which violate monotonicity prior to appending a new entry
+     */
+    private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) {
+        List<EpochEntry> removedEpochs = removeFromEnd(entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset);
+
+
+        if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) {
+
+            // Only log a warning if there were non-trivial removals. If the start offset of the new entry
+            // matches the start offset of the removed epoch, then no data has been written and the truncation
+            // is expected.
+            log.warn("New epoch entry {} caused truncation of conflicting entries {}. " + "Cache now contains {} entries.", newEntry, removedEpochs, epochs.size());
+        }
+    }
+
+    private List<EpochEntry> removeFromEnd(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeFromStart(Predicate<EpochEntry> predicate) {
+        return removeWhileMatching(epochs.entrySet().iterator(), predicate);
+    }
+
+    private List<EpochEntry> removeWhileMatching(Iterator<Map.Entry<Integer, EpochEntry>> iterator, Predicate<EpochEntry> predicate) {
+        ArrayList<EpochEntry> removedEpochs = new ArrayList<>();
+
+        while (iterator.hasNext()) {
+            EpochEntry entry = iterator.next().getValue();
+            if (predicate.test(entry)) {
+                removedEpochs.add(entry);
+                iterator.remove();
+            } else {
+                return removedEpochs;
+            }
+        }
+
+        return removedEpochs;
+    }
+
+    public boolean nonEmpty() {
+        lock.readLock().lock();
+        try {
+            return !epochs.isEmpty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public Optional<EpochEntry> latestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.lastEntry()).map(Map.Entry::getValue);
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the current Leader Epoch if one exists. This is the latest epoch
+     * which has messages assigned to it.
+     */
+    public OptionalInt latestEpoch() {
+        Optional<EpochEntry> entry = latestEntry();
+        return entry.isPresent() ? OptionalInt.of(entry.get().epoch) : OptionalInt.empty();
+    }
+
+    public OptionalInt previousEpoch() {
+        lock.readLock().lock();
+        try {
+            Optional<Map.Entry<Integer, EpochEntry>> lowerEntry = latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch)));
+            return lowerEntry.isPresent() ? OptionalInt.of(lowerEntry.get().getKey()) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Get the earliest cached entry if one exists.
+     */
+    public Optional<EpochEntry> earliestEntry() {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.firstEntry()).map(x -> x.getValue());
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt previousEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.lowerKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public OptionalInt nextEpoch(int epoch) {
+        lock.readLock().lock();
+        try {
+            return toOptionalInt(epochs.higherKey(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private static OptionalInt toOptionalInt(Integer value) {
+        return (value != null) ? OptionalInt.of(value) : OptionalInt.empty();
+    }
+
+    public Optional<EpochEntry> epochEntry(int epoch) {
+        lock.readLock().lock();
+        try {
+            return Optional.ofNullable(epochs.get(epoch));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns the Leader Epoch and the End Offset for a requested Leader Epoch.
+     * <p>
+     * The Leader Epoch returned is the largest epoch less than or equal to the requested Leader
+     * Epoch. The End Offset is the end offset of this epoch, which is defined as the start offset
+     * of the first Leader Epoch larger than the Leader Epoch requested, or else the Log End
+     * Offset if the latest epoch was requested.
+     * <p>
+     * During the upgrade phase, where there are existing messages may not have a leader epoch,
+     * if requestedEpoch is < the first epoch cached, UNDEFINED_EPOCH_OFFSET will be returned
+     * so that the follower falls back to High Water Mark.
+     *
+     * @param requestedEpoch requested leader epoch
+     * @param logEndOffset   the existing Log End Offset
+     * @return found leader epoch and end offset
+     */
+    public Map.Entry<Integer, Long> endOffsetFor(int requestedEpoch, long logEndOffset) {
+        lock.readLock().lock();
+        try {
+            Map.Entry<Integer, Long> epochAndOffset = null;
+            if (requestedEpoch == UNDEFINED_EPOCH) {
+                // This may happen if a bootstrapping follower sends a request with undefined epoch or
+                // a follower is on the older message format where leader epochs are not recorded
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+            } else if (latestEpoch().isPresent() && latestEpoch().getAsInt() == requestedEpoch) {
+                // For the leader, the latest epoch is always the current leader epoch that is still being written to.
+                // Followers should not have any reason to query for the end offset of the current epoch, but a consumer
+                // might if it is verifying its committed offset following a group rebalance. In this case, we return
+                // the current log end offset which makes the truncation check work as expected.
+                epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, logEndOffset);
+            } else {
+                Map.Entry<Integer, EpochEntry> higherEntry = epochs.higherEntry(requestedEpoch);
+                if (higherEntry == null) {
+                    // The requested epoch is larger than any known epoch. This case should never be hit because
+                    // the latest cached epoch is always the largest.
+                    epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET);
+                } else {
+                    Map.Entry<Integer, EpochEntry> floorEntry = epochs.floorEntry(requestedEpoch);
+                    if (floorEntry == null) {
+                        // The requested epoch is smaller than any known epoch, so we return the start offset of the first
+                        // known epoch which is larger than it. This may be inaccurate as there could have been
+                        // epochs in between, but the point is that the data has already been removed from the log
+                        // and we want to ensure that the follower can replicate correctly beginning from the leader's
+                        // start offset.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset);
+                    } else {
+                        // We have at least one previous epoch and one subsequent epoch. The result is the first
+                        // prior epoch and the starting offset of the first subsequent epoch.
+                        epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset);
+                    }
+                }
+            }
+
+            if (log.isTraceEnabled())
+                log.trace("Processed end offset request for epoch {} and returning epoch {} with end offset {} from epoch cache of size {}}", requestedEpoch, epochAndOffset.getKey(), epochAndOffset.getValue(), epochs.size());
+
+            return epochAndOffset;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
+     */
+    public void truncateFromEnd(long endOffset) {
+        lock.writeLock().lock();
+        try {
+            Optional<EpochEntry> epochEntry = latestEntry();
+            if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) {
+                List<EpochEntry> removedEntries = removeFromEnd(x -> x.startOffset >= endOffset);
+
+                flush();
+
+                log.debug("Cleared entries {} from epoch cache after truncating to end offset {}, leaving {} entries in the cache.", removedEntries, endOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
+     * be offset, then clears any previous epoch entries.
+     * <p>
+     * This method is exclusive: so truncateFromStart(6) will retain an entry at offset 6.
+     *
+     * @param startOffset the offset to clear up to
+     */
+    public void truncateFromStart(long startOffset) {
+        lock.writeLock().lock();
+        try {
+            List<EpochEntry> removedEntries = removeFromStart(entry -> entry.startOffset <= startOffset);
+
+            if (!removedEntries.isEmpty()) {
+                EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1);
+                EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset);
+                epochs.put(updatedFirstEntry.epoch, updatedFirstEntry);
+
+                flush();
+
+                log.debug("Cleared entries {} and rewrote first entry {} after truncating to start offset {}, leaving {} in the cache.", removedEntries, updatedFirstEntry, startOffset, epochs.size());
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public OptionalInt epochForOffset(long offset) {
+        lock.readLock().lock();
+        try {
+            Integer previousEpoch = null;
+            for (EpochEntry epochEntry : epochs.values()) {
+                int epoch = epochEntry.epoch;
+                long startOffset = epochEntry.startOffset;
+
+                // Found the exact offset, return the respective epoch.
+                if (startOffset == offset) return OptionalInt.of(epoch);
+
+                // exit from the loop as it is more than the target offset
+                if (startOffset > offset) break;
+
+                previousEpoch = epoch;
+            }
+
+            return previousEpoch != null ? OptionalInt.of(previousEpoch) : OptionalInt.empty();
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete all entries.
+     */
+    public void clearAndFlush() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+            flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void clear() {
+        lock.writeLock().lock();
+        try {
+            epochs.clear();
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    // Visible for testing
+    public List<EpochEntry> epochEntries() {
+        lock.writeLock().lock();
+        try {
+            return new ArrayList<>(epochs.values());
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public void flush() {
+        lock.writeLock().lock();

Review Comment:
   Let's go with that then as it is least likely to cause a change in behavior from what we had in the Scala code.



-- 
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] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1062119225


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LeaderEpochCheckpoint.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.internals;
+
+import java.util.List;
+
+public interface LeaderEpochCheckpoint {

Review Comment:
   I originally also preferred having `server` in the package name, but on further thought, there are advantages in basically having the module name as the package name after `kafka`. It avoids accidental split packages across modules, which can cause problems if we ever decide to use Java 9 modules. And it means that a given server module could eventually be used by clients too if it made sense (without forcing package renames or weird package names).
   
   If we agree on that approach, then the module names would be `org.apache.kafka.storage.internals.checkpoint` and `org.apache.kafka.storage.internals.log` (one annoying aspect of using `log` is that GitHub filters it from their code navigation, but we probably have to live with that for now).
   
   @junrao what are your 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] ijuma commented on a diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
ijuma commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1070638073


##########
server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java:
##########
@@ -72,7 +71,7 @@ public CheckpointFile(File file,
         tempPath = Paths.get(absolutePath.toString() + ".tmp");
     }
 
-    public void write(Collection<T> entries) throws IOException {
+    public void write(List<T> entries) throws IOException {

Review Comment:
   I don't think consistency is necessarily a goal in cases like this. The original intent was to accept the minimal interface required, but to return something more specific to avoid unnecessary conversions.



-- 
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 diff in pull request #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

Posted by GitBox <gi...@apache.org>.
satishd commented on code in PR #13046:
URL: https://github.com/apache/kafka/pull/13046#discussion_r1070596038


##########
server-common/src/main/java/org/apache/kafka/server/common/CheckpointFile.java:
##########
@@ -72,7 +71,7 @@ public CheckpointFile(File file,
         tempPath = Paths.get(absolutePath.toString() + ".tmp");
     }
 
-    public void write(Collection<T> entries) throws IOException {
+    public void write(List<T> entries) throws IOException {

Review Comment:
   This is done to make write and read API consistent like below.
   
   ```
   public void write(List<T> entries) throws IOException 
   public List<T> read() throws IOException 
   ```



-- 
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 #13046: KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

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

   Thanks @ijuma for your review. Addressed them with inline comments and/or updated commits.


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