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

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

Reply via email to