satishd commented on a change in pull request #10218: URL: https://github.com/apache/kafka/pull/10218#discussion_r607559905
########## File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java ########## @@ -0,0 +1,331 @@ +/* + * 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.remote.storage; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments + * with respect to leader epochs. + * <p> + * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}. + * <p> + * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any + * segment reaching the terminal state will get cleared from this instance. + * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)}, + * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those + * methods have different semantics to fetch the segment based on its state. + * <p> + * <ul> + * <li> + * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}: + * <br> + * Segment in this state indicate it is not yet copied successfully. So, these segments will not be + * accessible for reads but these are considered for cleanups when a partition is deleted. + * </li> + * <li> + * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}: + * <br> + * Segment in this state indicate it is successfully copied and it is available for reads. So, these segments + * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the + * caller of this class. + * </li> + * <li> + * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}: + * Segment in this state indicate it is getting deleted. That means, it is not available for reads. But it should be + * available for any cleanup activity like deleting segments by the caller of this class. + * </li> + * <li> + * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}: + * Segment in this state indicate it is already deleted. That means, it is not available for any activity including + * reads or cleanup activity. This cache will clear entries containing this state. + * </li> + * </ul> + * + * <p> + * <table border="4"> + * <thead border="4"> + * <tr> + * <th></th> + * <th>COPY_SEGMENT_STARTED</th> + * <th>COPY_SEGMENT_FINISHED</th> + * <th>DELETE_SEGMENT_STARTED</th> + * <th>DELETE_SEGMENT_FINISHED</th> + * </tr> + * </thead> + * <tbody> + * <tr> + * <td>remoteLogSegmentMetadata<br>(int leaderEpoch, long offset)</td> + * <td>No</td> + * <td>Yes</td> + * <td>No</td> + * <td>No</td> + * </tr> + * <tr> + * <td>listRemoteLogSegments <br>(int leaderEpoch)</td> + * <td>Yes</td> + * <td>Yes</td> + * <td>Yes</td> + * <td>No</td> + * </tr> + * <tr> + * <td>listAllRemoteLogSegments()</td> + * <td>Yes</td> + * <td>Yes</td> + * <td>Yes</td> + * <td>No</td> + * </tr> + * </tbody> + * </table> + * </p> + * <p></p> + */ +public class RemoteLogMetadataCache { + + private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class); + + // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED. + private final ConcurrentMap<RemoteLogSegmentId, RemoteLogSegmentMetadata> idToSegmentMetadata + = new ConcurrentHashMap<>(); + + // It contains leader epoch to the respective entry containing the state. + private final ConcurrentMap<Integer, RemoteLogLeaderEpochState> leaderEpochEntries = new ConcurrentHashMap<>(); + + /** + * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with + * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}. + * + * @param leaderEpoch leader epoch for the given offset + * @param offset offset + * @return the requested remote log segment metadata if it exists. + */ + public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(int leaderEpoch, long offset) { + RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch); + + if (remoteLogLeaderEpochState == null) { + return Optional.empty(); + } + + // Look for floor entry as the given offset may exist in this entry. + RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset); + if (remoteLogSegmentId == null) { + // If the offset is lower than the minimum offset available in metadata then return empty. + return Optional.empty(); + } + + RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId); + // Check whether the given offset with leaderEpoch exists in this segment. + // Check for epoch's offset boundaries with in this segment. + // 1. Get the next epoch's start offset -1 if exists + // 2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset. + Map.Entry<Integer, Long> nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch); + long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset(); + + // Return empty when target offset > epoch's end offset or segment is not in COPY_SEGMENT_FINISHED state. + // This segment will not be available in offsetToId when it reaches the DELETE_SEGMENT_FINISHED state. So, no + // need to add for that state here. + return (offset > epochEndOffset || metadata.state() != RemoteLogSegmentState.COPY_SEGMENT_FINISHED) Review comment: good point, this check is no more needed. -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org