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 2021/04/06 16:59:31 UTC

[GitHub] [kafka] kowshik commented on a change in pull request #10218: KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

kowshik commented on a change in pull request #10218:
URL: https://github.com/apache/kafka/pull/10218#discussion_r607959239



##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,48 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     * @return

Review comment:
       nit: remove empty `@return`

##########
File path: clients/src/test/java/org/apache/kafka/test/TestUtils.java
##########
@@ -535,4 +536,48 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr
         field.setAccessible(true);
         field.set(obj, value);
     }
+
+    /**
+     * Returns true if both iterators have same elements in the same order.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     * @return
+     */
+    public static <T> boolean sameElementsWithOrder(Iterator<T> iterator1,
+                                                    Iterator<T> iterator2) {
+        while (iterator1.hasNext()) {
+            if (!iterator2.hasNext()) {
+                return false;
+            }
+
+            Object elem1 = iterator1.next();
+            Object elem2 = iterator2.next();
+            if (!Objects.equals(elem1, elem2)) {
+                return false;
+            }
+        }
+
+        return !iterator2.hasNext();
+    }
+
+    /**
+     * Returns true if both the iterators have same set of elements irrespective of order and duplicates.
+     *
+     * @param iterator1 first iterator.
+     * @param iterator2 second iterator.
+     * @param <T>       type of element in the iterators.
+     * @return

Review comment:
       nit: remove empty `@return`

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java
##########
@@ -87,4 +89,27 @@ public byte id() {
     public static RemoteLogSegmentState forId(byte id) {
         return STATE_TYPES.get(id);
     }
+
+    public static boolean isValidTransition(RemoteLogSegmentState srcState, RemoteLogSegmentState targetState) {
+        Objects.requireNonNull(targetState, "targetState can not be null");
+
+        if (srcState == null) {

Review comment:
       Same comment as before: https://github.com/apache/kafka/pull/10218/files#r598982742.
   Can srcState be null in practice? If not, this can be defined as an instance method.

##########
File path: remote-storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataCache.java
##########
@@ -0,0 +1,305 @@
+/*
+ * 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 indicates 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 indicates 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 indicates 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>
+ * <pre>
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * |                                 | COPY_SEGMENT_STARTED | COPY_SEGMENT_FINISHED  | DELETE_SEGMENT_STARTED  | DELETE_SEGMENT_STARTED  |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | remoteLogSegmentMetadata        |        No            |           Yes          |          No             |           No            |
+ * | (int leaderEpoch, long offset)  |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listRemoteLogSegments           |        Yes           |           Yes          |          Yes            |           No            |
+ * | (int leaderEpoch)               |                      |                        |                         |                         |
+ * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
+ * | listAllRemoteLogSegments()      |        Yes           |           Yes          |          Yes            |           No            |
+ * |                                 |                      |                        |                         |                         |
+ * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
+ * </pre>
+ * </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<>();

Review comment:
       Should we call this map as `idToLeaderEpochState` or `idToEpochState` similar to the naming for the other map?




-- 
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:
users@infra.apache.org