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/03/02 01:24:09 UTC

[GitHub] [kafka] junrao commented on a change in pull request #10173: KAFKA-9548 Added SPIs and public classes/interfaces introduced in KIP-405 for tiered storage feature in Kafka.

junrao commented on a change in pull request #10173:
URL: https://github.com/apache/kafka/pull/10173#discussion_r585109788



##########
File path: clients/src/main/java/org/apache/kafka/common/TopicIdPartition.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.common;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * This represents universally unique identifier with topic id for a topic partition. This makes sure that topics
+ * recreated with the same name will always have unique topic identifiers.
+ */
+public class TopicIdPartition implements Serializable {

Review comment:
       Could we consolidate this with the TopicIdPartition in BrokersToIsrs?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * It describes the metadata about a topic partition's remote log segment in the remote storage. This is uniquely
+ * represented with {@link RemoteLogSegmentId}.
+ * <p>
+ * New instance is always created with the state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}. This can be
+ * updated by applying {@link RemoteLogSegmentMetadataUpdate} for the respective {@link RemoteLogSegmentId} of the
+ * {@code RemoteLogSegmentMetadata}.
+ */
+@InterfaceStability.Evolving
+public class RemoteLogSegmentMetadata implements Serializable {

Review comment:
       We typically don't use java serialization. Is Serializable needed? Ditto in a few other classes.

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentId.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * This class represents a universally unique identifier associated to a topic partition's log segment. This will be
+ * regenerated for every attempt of copying a specific log segment in {@link RemoteStorageManager#copyLogSegmentData(RemoteLogSegmentMetadata, LogSegmentData)}.
+ * Once it is stored in remote storage, it is used to access that segment later from remote log metadata storage.
+ */
+@InterfaceStability.Evolving
+public class RemoteLogSegmentId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final TopicIdPartition topicIdPartition;
+    private final UUID id;

Review comment:
       We now have a Uuid class. Should we use that?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * It describes the metadata about a topic partition's remote log segment in the remote storage. This is uniquely
+ * represented with {@link RemoteLogSegmentId}.
+ * <p>
+ * New instance is always created with the state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}. This can be
+ * updated by applying {@link RemoteLogSegmentMetadataUpdate} for the respective {@link RemoteLogSegmentId} of the
+ * {@code RemoteLogSegmentMetadata}.
+ */
+@InterfaceStability.Evolving
+public class RemoteLogSegmentMetadata implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Universally unique remote log segment id.
+     */
+    private final RemoteLogSegmentId remoteLogSegmentId;
+
+    /**
+     * Start offset of this segment.
+     */
+    private final long startOffset;
+
+    /**
+     * End offset of this segment.
+     */
+    private final long endOffset;
+
+    /**
+     * Broker id from which this event is generated.
+     */
+    private final int brokerId;
+
+    /**
+     * Maximum timestamp in the segment
+     */
+    private final long maxTimestamp;
+
+    /**
+     * Epoch time at which the respective {@link #state} is set.
+     */
+    private final long eventTimestamp;
+
+    /**
+     * LeaderEpoch vs offset for messages with in this segment.
+     */
+    private final NavigableMap<Integer, Long> segmentLeaderEpochs;
+
+    /**
+     * Size of the segment in bytes.
+     */
+    private final int segmentSizeInBytes;
+
+    /**
+     * It indicates the state in which the action is executed on this segment.
+     */
+    private final RemoteLogSegmentState state;
+
+    /**
+     * Creates an instance with the given metadata of remote log segment.
+     *
+     * {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
+     * then it should have an entry with epoch mapping to start-offset of this segment.
+     *
+     * @param remoteLogSegmentId  Universally unique remote log segment id.
+     * @param startOffset         Start offset of this segment.
+     * @param endOffset           End offset of this segment.
+     * @param maxTimestamp        Maximum timestamp in this segment.
+     * @param brokerId            Broker id from which this event is generated.
+     * @param eventTimestamp      Epoch time at which the remote log segment is copied to the remote tier storage.
+     * @param segmentSizeInBytes  Size of this segment in bytes.
+     * @param state               State of the respective segment of remoteLogSegmentId.
+     * @param segmentLeaderEpochs leader epochs occurred with in this segment.
+     */
+    private RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId,
+                                     long startOffset,
+                                     long endOffset,
+                                     long maxTimestamp,
+                                     int brokerId,
+                                     long eventTimestamp,
+                                     int segmentSizeInBytes,
+                                     RemoteLogSegmentState state,
+                                     Map<Integer, Long> segmentLeaderEpochs) {
+        this.remoteLogSegmentId = Objects.requireNonNull(remoteLogSegmentId, "remoteLogSegmentId can not be null");
+        this.state = Objects.requireNonNull(state, "state can not be null");
+
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+        this.brokerId = brokerId;
+        this.maxTimestamp = maxTimestamp;
+        this.eventTimestamp = eventTimestamp;
+        this.segmentSizeInBytes = segmentSizeInBytes;
+
+        if (segmentLeaderEpochs == null || segmentLeaderEpochs.isEmpty()) {
+            throw new IllegalArgumentException("segmentLeaderEpochs can not be null or empty");
+        }
+
+        this.segmentLeaderEpochs = Collections
+                .unmodifiableNavigableMap(new ConcurrentSkipListMap<>(segmentLeaderEpochs));
+    }
+
+    /**
+     * Creates an instance with the given metadata of remote log segment and it's state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}.

Review comment:
       it's  => its 

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * It describes the metadata about a topic partition's remote log segment in the remote storage. This is uniquely
+ * represented with {@link RemoteLogSegmentId}.
+ * <p>
+ * New instance is always created with the state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}. This can be
+ * updated by applying {@link RemoteLogSegmentMetadataUpdate} for the respective {@link RemoteLogSegmentId} of the
+ * {@code RemoteLogSegmentMetadata}.
+ */
+@InterfaceStability.Evolving
+public class RemoteLogSegmentMetadata implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Universally unique remote log segment id.
+     */
+    private final RemoteLogSegmentId remoteLogSegmentId;
+
+    /**
+     * Start offset of this segment.
+     */
+    private final long startOffset;
+
+    /**
+     * End offset of this segment.
+     */
+    private final long endOffset;
+
+    /**
+     * Broker id from which this event is generated.
+     */
+    private final int brokerId;
+
+    /**
+     * Maximum timestamp in the segment
+     */
+    private final long maxTimestamp;
+
+    /**
+     * Epoch time at which the respective {@link #state} is set.
+     */
+    private final long eventTimestamp;
+
+    /**
+     * LeaderEpoch vs offset for messages with in this segment.
+     */
+    private final NavigableMap<Integer, Long> segmentLeaderEpochs;
+
+    /**
+     * Size of the segment in bytes.
+     */
+    private final int segmentSizeInBytes;
+
+    /**
+     * It indicates the state in which the action is executed on this segment.
+     */
+    private final RemoteLogSegmentState state;
+
+    /**
+     * Creates an instance with the given metadata of remote log segment.
+     *
+     * {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
+     * then it should have an entry with epoch mapping to start-offset of this segment.
+     *
+     * @param remoteLogSegmentId  Universally unique remote log segment id.
+     * @param startOffset         Start offset of this segment.
+     * @param endOffset           End offset of this segment.
+     * @param maxTimestamp        Maximum timestamp in this segment.
+     * @param brokerId            Broker id from which this event is generated.
+     * @param eventTimestamp      Epoch time at which the remote log segment is copied to the remote tier storage.
+     * @param segmentSizeInBytes  Size of this segment in bytes.
+     * @param state               State of the respective segment of remoteLogSegmentId.
+     * @param segmentLeaderEpochs leader epochs occurred with in this segment.
+     */
+    private RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId,
+                                     long startOffset,
+                                     long endOffset,
+                                     long maxTimestamp,
+                                     int brokerId,
+                                     long eventTimestamp,
+                                     int segmentSizeInBytes,
+                                     RemoteLogSegmentState state,
+                                     Map<Integer, Long> segmentLeaderEpochs) {
+        this.remoteLogSegmentId = Objects.requireNonNull(remoteLogSegmentId, "remoteLogSegmentId can not be null");
+        this.state = Objects.requireNonNull(state, "state can not be null");
+
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+        this.brokerId = brokerId;
+        this.maxTimestamp = maxTimestamp;
+        this.eventTimestamp = eventTimestamp;
+        this.segmentSizeInBytes = segmentSizeInBytes;
+
+        if (segmentLeaderEpochs == null || segmentLeaderEpochs.isEmpty()) {
+            throw new IllegalArgumentException("segmentLeaderEpochs can not be null or empty");
+        }
+
+        this.segmentLeaderEpochs = Collections
+                .unmodifiableNavigableMap(new ConcurrentSkipListMap<>(segmentLeaderEpochs));
+    }
+
+    /**
+     * Creates an instance with the given metadata of remote log segment and it's state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}.
+     *
+     * {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
+     * then it should have an entry with epoch mapping to start-offset of this segment.
+     *
+     * @param remoteLogSegmentId  Universally unique remote log segment id.
+     * @param startOffset         Start offset of this segment.
+     * @param endOffset           End offset of this segment.
+     * @param maxTimestamp        Maximum timestamp in this segment
+     * @param brokerId            Broker id from which this event is generated.
+     * @param eventTimestamp      Epoch time at which the remote log segment is copied to the remote tier storage.
+     * @param segmentSizeInBytes  Size of this segment in bytes.
+     * @param segmentLeaderEpochs leader epochs occurred with in this segment
+     */
+    public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId,
+                                    long startOffset,
+                                    long endOffset,
+                                    long maxTimestamp,
+                                    int brokerId,
+                                    long eventTimestamp,
+                                    int segmentSizeInBytes,
+                                    Map<Integer, Long> segmentLeaderEpochs) {
+        this.remoteLogSegmentId = remoteLogSegmentId;
+        this.startOffset = startOffset;
+        this.endOffset = endOffset;
+        this.brokerId = brokerId;
+        this.maxTimestamp = maxTimestamp;
+        this.eventTimestamp = eventTimestamp;
+        this.state = RemoteLogSegmentState.COPY_SEGMENT_STARTED;
+        this.segmentSizeInBytes = segmentSizeInBytes;
+        this.segmentLeaderEpochs = Collections
+                .unmodifiableNavigableMap(new ConcurrentSkipListMap<>(segmentLeaderEpochs));
+    }
+
+
+    /**
+     * @return unique id of this segment.
+     */
+    public RemoteLogSegmentId remoteLogSegmentId() {
+        return remoteLogSegmentId;
+    }
+
+    /**
+     * @return Start offset of this segment(inclusive).
+     */
+    public long startOffset() {
+        return startOffset;
+    }
+
+    /**
+     * @return End offset of this segment(inclusive).
+     */
+    public long endOffset() {
+        return endOffset;
+    }
+
+    /**
+     * @return Epoch time at which this event is occurred.
+     */
+    public long eventTimestamp() {
+        return eventTimestamp;
+    }
+
+    /**
+     * @return Total size of this segment in bytes.
+     */
+    public int segmentSizeInBytes() {
+        return segmentSizeInBytes;
+    }
+
+    /**
+     * @return Maximum timestamp of a record with in this segment.

Review comment:
       with in => within

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+/**
+ * It describes the metadata about a topic partition's remote log segment in the remote storage. This is uniquely
+ * represented with {@link RemoteLogSegmentId}.
+ * <p>
+ * New instance is always created with the state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}. This can be
+ * updated by applying {@link RemoteLogSegmentMetadataUpdate} for the respective {@link RemoteLogSegmentId} of the
+ * {@code RemoteLogSegmentMetadata}.
+ */
+@InterfaceStability.Evolving
+public class RemoteLogSegmentMetadata implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Universally unique remote log segment id.
+     */
+    private final RemoteLogSegmentId remoteLogSegmentId;
+
+    /**
+     * Start offset of this segment.
+     */
+    private final long startOffset;
+
+    /**
+     * End offset of this segment.
+     */
+    private final long endOffset;
+
+    /**
+     * Broker id from which this event is generated.
+     */
+    private final int brokerId;
+
+    /**
+     * Maximum timestamp in the segment
+     */
+    private final long maxTimestamp;
+
+    /**
+     * Epoch time at which the respective {@link #state} is set.
+     */
+    private final long eventTimestamp;
+
+    /**
+     * LeaderEpoch vs offset for messages with in this segment.
+     */
+    private final NavigableMap<Integer, Long> segmentLeaderEpochs;
+
+    /**
+     * Size of the segment in bytes.
+     */
+    private final int segmentSizeInBytes;
+
+    /**
+     * It indicates the state in which the action is executed on this segment.
+     */
+    private final RemoteLogSegmentState state;
+
+    /**
+     * Creates an instance with the given metadata of remote log segment.
+     *
+     * {@code segmentLeaderEpochs} can not be empty. If all the records in this segment belong to the same leader epoch
+     * then it should have an entry with epoch mapping to start-offset of this segment.
+     *
+     * @param remoteLogSegmentId  Universally unique remote log segment id.
+     * @param startOffset         Start offset of this segment.
+     * @param endOffset           End offset of this segment.

Review comment:
       Could we document whether this is inclusive or exclusive?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * It describes the metadata update about the log segment in the remote storage. This is currently used to update the
+ * state of the remote log segment by using {@link RemoteLogMetadataManager#updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate)}.
+ * This also includes the timestamp of this event.
+ */
+@InterfaceStability.Evolving
+public class RemoteLogSegmentMetadataUpdate implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Universally unique remote log segment id.
+     */
+    private final RemoteLogSegmentId remoteLogSegmentId;
+
+    /**
+     * Epoch time at which this event is generated.
+     */
+    private final long eventTimestamp;
+
+    /**
+     * It indicates the state in which the action is executed on this segment.
+     */
+    private final RemoteLogSegmentState state;
+
+    /**
+     * Broker id from which this event is generated.
+     */
+    private final int brokerId;
+
+    /**
+     * @param remoteLogSegmentId Universally unique remote log segment id.
+     * @param eventTimestamp     Epoch time at which the remote log segment is copied to the remote tier storage.

Review comment:
       It would be useful to document the epoch granularity.

##########
File path: clients/src/main/java/org/apache/kafka/common/TopicIdPartition.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.common;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * This represents universally unique identifier with topic id for a topic partition. This makes sure that topics
+ * recreated with the same name will always have unique topic identifiers.
+ */
+public class TopicIdPartition implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final UUID topicId;

Review comment:
       We now have a Uuid class. Should we use that?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.apache.kafka.common.Configurable;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Closeable;
+import java.io.InputStream;
+
+/**
+ * This interface provides the lifecycle of remote log segments that includes copy, fetch, and delete from remote
+ * storage.
+ * <p>
+ * Each upload or copy of a segment is initiated with {@link RemoteLogSegmentMetadata} containing {@link RemoteLogSegmentId}
+ * which is universally unique even for the same topic partition and offsets.
+ * <p>
+ * {@link RemoteLogSegmentMetadata} is stored in {@link RemoteLogMetadataManager} before and after copy/delete operations on
+ * {@link RemoteStorageManager} with the respective {@link RemoteLogSegmentState}. {@link RemoteLogMetadataManager} is
+ * responsible for storing and fetching metadata about the remote log segments in a strongly consistent manner.
+ * This allows {@link RemoteStorageManager} to store segments even in eventually consistent manner as the metadata is already

Review comment:
       We do expect RemoteStorageManager to have strong consistency on the data. We only relax the requirements on metadata consistency. So, it would be useful to make this clear.

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.apache.kafka.common.Configurable;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Closeable;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This interface provides storing and fetching remote log segment metadata with strongly consistent semantics.
+ * <p>
+ * This class can be plugged in to Kafka cluster by adding the implementation class as
+ * <code>remote.log.metadata.manager.class.name</code> property value. There is an inbuilt implementation backed by
+ * topic storage in the local cluster. This is used as the default implementation if
+ * remote.log.metadata.manager.class.name is not configured.
+ * </p>
+ * <p>
+ * <code>remote.log.metadata.manager.class.path</code> property is about the class path of the RemoteLogStorageManager
+ * implementation. If specified, the RemoteLogStorageManager implementation and its dependent libraries will be loaded
+ * by a dedicated classloader which searches this class path before the Kafka broker class path. The syntax of this
+ * parameter is same with the standard Java class path string.
+ * </p>
+ * <p>
+ * <code>remote.log.metadata.manager.listener.name</code> property is about listener name of the local broker to which
+ * it should get connected if needed by RemoteLogMetadataManager implementation. When this is configured all other
+ * required properties can be passed as properties with prefix of 'remote.log.metadata.manager.listener.
+ * </p>
+ * "cluster.id", "broker.id" and all other properties prefixed with "remote.log.metadata." are passed when
+ * {@link #configure(Map)} is invoked on this instance.
+ * <p>
+ */
+@InterfaceStability.Evolving
+public interface RemoteLogMetadataManager extends Configurable, Closeable {
+
+    /**
+     * Stores {@link }RemoteLogSegmentMetadata} with the containing {@link }RemoteLogSegmentId} into {@link RemoteLogMetadataManager}.
+     * <p>
+     * RemoteLogSegmentMetadata is identified by RemoteLogSegmentId.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    void putRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException;
+
+    /**
+     * This method is used to update the {@link RemoteLogSegmentMetadata}. Currently, it allows to update with the new
+     * state based on the life cycle of the segment. It can go through the below state transitions.
+     * <p>
+     * <pre>
+     * +---------------------+            +----------------------+
+     * |COPY_SEGMENT_STARTED |----------->|COPY_SEGMENT_FINISHED |
+     * +-------------------+-+            +--+-------------------+
+     *                     |                 |
+     *                     |                 |
+     *                     v                 v
+     *                  +--+-----------------+-+
+     *                  |DELETE_SEGMENT_STARTED|
+     *                  +-----------+----------+
+     *                              |
+     *                              |
+     *                              v
+     *                  +-----------+-----------+
+     *                  |DELETE_SEGMENT_FINISHED|
+     *                  +-----------------------+
+     * </pre>
+     * <p>
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED} - This state indicates that the segment copying to remote storage is started but not yet finished.
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} - This state indicates that the segment copying to remote storage is finished.
+     * <br>
+     * The leader broker copies the log segments to the remote storage and puts the remote log segment metadata with the
+     * state as “COPY_SEGMENT_STARTED” and updates the state as “COPY_SEGMENT_FINISHED” once the copy is successful.
+     * <p></p>
+     * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED} - This state indicates that the segment deletion is started but not yet finished.
+     * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED} - This state indicates that the segment is deleted successfully.
+     * <br>
+     * Leader partitions publish both the above delete segment events when remote log retention is reached for the
+     * respective segments. Remote Partition Removers also publish these events when a segment is deleted as part of
+     * the remote partition deletion.
+     *
+     * @param remoteLogSegmentMetadataUpdate update of the remote log segment metadata.
+     * @throws RemoteStorageException          if there are any storage related errors occurred.
+     * @throws RemoteResourceNotFoundException when there are no resources associated with the given remoteLogSegmentMetadataUpdate.
+     */
+    void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate remoteLogSegmentMetadataUpdate)
+            throws RemoteStorageException;
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given topic partition containing the offset with
+     * the given leader-epoch for the offset, else returns {@link Optional#empty()}.
+     *
+     * @param topicIdPartition topic partition
+     * @param offset           offset
+     * @param epochForOffset   leader epoch for the given offset
+     * @return the requested remote log segment metadata if it exists.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                long offset,
+                                                                int epochForOffset)
+            throws RemoteStorageException;
+
+    /**
+     * Returns the highest log offset of topic partition for the given leader epoch in remote storage. This is used by
+     * remote log management subsystem to know upto which offset the segments have been copied to remote storage for
+     * a given leader epoch.
+     *
+     * @param topicIdPartition topic partition
+     * @param leaderEpoch      leader epoch
+     * @return the requested highest log offset if exists.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    Optional<Long> highestLogOffset(TopicIdPartition topicIdPartition,
+                                    int leaderEpoch) throws RemoteStorageException;
+
+    /**
+     * This method is used to update the metadata about remote partition delete event. Currently, it allows updating the
+     * state ({@link RemotePartitionDeleteState}) of a topic partition in remote metadata storage. Controller invokes
+     * this method with {@link RemotePartitionDeleteMetadata} having state as {@link RemotePartitionDeleteState#DELETE_PARTITION_MARKED}.
+     * So, remote partition removers can act on this event to clean the respective remote log segments of the partition.
+     * <p><br>
+     * Incase of default RLMM implementation, remote partition remover processes {@link RemotePartitionDeleteState#DELETE_PARTITION_MARKED}

Review comment:
       Incase => In the case

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.apache.kafka.common.Configurable;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Closeable;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This interface provides storing and fetching remote log segment metadata with strongly consistent semantics.
+ * <p>
+ * This class can be plugged in to Kafka cluster by adding the implementation class as
+ * <code>remote.log.metadata.manager.class.name</code> property value. There is an inbuilt implementation backed by
+ * topic storage in the local cluster. This is used as the default implementation if
+ * remote.log.metadata.manager.class.name is not configured.
+ * </p>
+ * <p>
+ * <code>remote.log.metadata.manager.class.path</code> property is about the class path of the RemoteLogStorageManager
+ * implementation. If specified, the RemoteLogStorageManager implementation and its dependent libraries will be loaded
+ * by a dedicated classloader which searches this class path before the Kafka broker class path. The syntax of this
+ * parameter is same with the standard Java class path string.
+ * </p>
+ * <p>
+ * <code>remote.log.metadata.manager.listener.name</code> property is about listener name of the local broker to which
+ * it should get connected if needed by RemoteLogMetadataManager implementation. When this is configured all other
+ * required properties can be passed as properties with prefix of 'remote.log.metadata.manager.listener.
+ * </p>
+ * "cluster.id", "broker.id" and all other properties prefixed with "remote.log.metadata." are passed when
+ * {@link #configure(Map)} is invoked on this instance.
+ * <p>
+ */
+@InterfaceStability.Evolving
+public interface RemoteLogMetadataManager extends Configurable, Closeable {
+
+    /**
+     * Stores {@link }RemoteLogSegmentMetadata} with the containing {@link }RemoteLogSegmentId} into {@link RemoteLogMetadataManager}.
+     * <p>
+     * RemoteLogSegmentMetadata is identified by RemoteLogSegmentId.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    void putRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException;
+
+    /**
+     * This method is used to update the {@link RemoteLogSegmentMetadata}. Currently, it allows to update with the new
+     * state based on the life cycle of the segment. It can go through the below state transitions.
+     * <p>
+     * <pre>
+     * +---------------------+            +----------------------+
+     * |COPY_SEGMENT_STARTED |----------->|COPY_SEGMENT_FINISHED |
+     * +-------------------+-+            +--+-------------------+
+     *                     |                 |
+     *                     |                 |
+     *                     v                 v
+     *                  +--+-----------------+-+
+     *                  |DELETE_SEGMENT_STARTED|
+     *                  +-----------+----------+
+     *                              |
+     *                              |
+     *                              v
+     *                  +-----------+-----------+
+     *                  |DELETE_SEGMENT_FINISHED|
+     *                  +-----------------------+
+     * </pre>
+     * <p>
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED} - This state indicates that the segment copying to remote storage is started but not yet finished.
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} - This state indicates that the segment copying to remote storage is finished.
+     * <br>
+     * The leader broker copies the log segments to the remote storage and puts the remote log segment metadata with the
+     * state as “COPY_SEGMENT_STARTED” and updates the state as “COPY_SEGMENT_FINISHED” once the copy is successful.
+     * <p></p>
+     * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED} - This state indicates that the segment deletion is started but not yet finished.
+     * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED} - This state indicates that the segment is deleted successfully.
+     * <br>
+     * Leader partitions publish both the above delete segment events when remote log retention is reached for the
+     * respective segments. Remote Partition Removers also publish these events when a segment is deleted as part of
+     * the remote partition deletion.
+     *
+     * @param remoteLogSegmentMetadataUpdate update of the remote log segment metadata.
+     * @throws RemoteStorageException          if there are any storage related errors occurred.
+     * @throws RemoteResourceNotFoundException when there are no resources associated with the given remoteLogSegmentMetadataUpdate.
+     */
+    void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate remoteLogSegmentMetadataUpdate)
+            throws RemoteStorageException;
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given topic partition containing the offset with
+     * the given leader-epoch for the offset, else returns {@link Optional#empty()}.
+     *
+     * @param topicIdPartition topic partition
+     * @param offset           offset
+     * @param epochForOffset   leader epoch for the given offset
+     * @return the requested remote log segment metadata if it exists.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                long offset,
+                                                                int epochForOffset)
+            throws RemoteStorageException;
+
+    /**
+     * Returns the highest log offset of topic partition for the given leader epoch in remote storage. This is used by
+     * remote log management subsystem to know upto which offset the segments have been copied to remote storage for
+     * a given leader epoch.
+     *
+     * @param topicIdPartition topic partition
+     * @param leaderEpoch      leader epoch
+     * @return the requested highest log offset if exists.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    Optional<Long> highestLogOffset(TopicIdPartition topicIdPartition,
+                                    int leaderEpoch) throws RemoteStorageException;
+
+    /**
+     * This method is used to update the metadata about remote partition delete event. Currently, it allows updating the
+     * state ({@link RemotePartitionDeleteState}) of a topic partition in remote metadata storage. Controller invokes
+     * this method with {@link RemotePartitionDeleteMetadata} having state as {@link RemotePartitionDeleteState#DELETE_PARTITION_MARKED}.
+     * So, remote partition removers can act on this event to clean the respective remote log segments of the partition.
+     * <p><br>
+     * Incase of default RLMM implementation, remote partition remover processes {@link RemotePartitionDeleteState#DELETE_PARTITION_MARKED}
+     * <ul>
+     * <li> sends an event with state as {@link RemotePartitionDeleteState#DELETE_PARTITION_STARTED}
+     * <li> gets all the remote log segments and deletes them.
+     * <li> sends an event with state as {@link RemotePartitionDeleteState#DELETE_PARTITION_FINISHED} once all the remote log segments are
+     * deleted.
+     * </ul>
+     *
+     * @param remotePartitionDeleteMetadata update on delete state of a partition.
+     * @throws RemoteStorageException          if there are any storage related errors occurred.
+     * @throws RemoteResourceNotFoundException when there are no resources associated with the given remotePartitionDeleteMetadata.
+     */
+    void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata)
+            throws RemoteStorageException;
+
+    /**
+     * List all the remote log segment metadata of the given topicIdPartition.
+     * <p>
+     * Remote Partition Removers uses this method to fetch all the segments for a given topic partition, so that they
+     * can delete them.
+     *
+     * @return Iterator of remote log segment metadata for the given topic partition.

Review comment:
       Is the result sorted in any way?

##########
File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.apache.kafka.common.Configurable;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.io.Closeable;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This interface provides storing and fetching remote log segment metadata with strongly consistent semantics.
+ * <p>
+ * This class can be plugged in to Kafka cluster by adding the implementation class as
+ * <code>remote.log.metadata.manager.class.name</code> property value. There is an inbuilt implementation backed by
+ * topic storage in the local cluster. This is used as the default implementation if
+ * remote.log.metadata.manager.class.name is not configured.
+ * </p>
+ * <p>
+ * <code>remote.log.metadata.manager.class.path</code> property is about the class path of the RemoteLogStorageManager
+ * implementation. If specified, the RemoteLogStorageManager implementation and its dependent libraries will be loaded
+ * by a dedicated classloader which searches this class path before the Kafka broker class path. The syntax of this
+ * parameter is same with the standard Java class path string.
+ * </p>
+ * <p>
+ * <code>remote.log.metadata.manager.listener.name</code> property is about listener name of the local broker to which
+ * it should get connected if needed by RemoteLogMetadataManager implementation. When this is configured all other
+ * required properties can be passed as properties with prefix of 'remote.log.metadata.manager.listener.
+ * </p>
+ * "cluster.id", "broker.id" and all other properties prefixed with "remote.log.metadata." are passed when
+ * {@link #configure(Map)} is invoked on this instance.
+ * <p>
+ */
+@InterfaceStability.Evolving
+public interface RemoteLogMetadataManager extends Configurable, Closeable {
+
+    /**
+     * Stores {@link }RemoteLogSegmentMetadata} with the containing {@link }RemoteLogSegmentId} into {@link RemoteLogMetadataManager}.
+     * <p>
+     * RemoteLogSegmentMetadata is identified by RemoteLogSegmentId.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    void putRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException;
+
+    /**
+     * This method is used to update the {@link RemoteLogSegmentMetadata}. Currently, it allows to update with the new
+     * state based on the life cycle of the segment. It can go through the below state transitions.
+     * <p>
+     * <pre>
+     * +---------------------+            +----------------------+
+     * |COPY_SEGMENT_STARTED |----------->|COPY_SEGMENT_FINISHED |
+     * +-------------------+-+            +--+-------------------+
+     *                     |                 |
+     *                     |                 |
+     *                     v                 v
+     *                  +--+-----------------+-+
+     *                  |DELETE_SEGMENT_STARTED|
+     *                  +-----------+----------+
+     *                              |
+     *                              |
+     *                              v
+     *                  +-----------+-----------+
+     *                  |DELETE_SEGMENT_FINISHED|
+     *                  +-----------------------+
+     * </pre>
+     * <p>
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED} - This state indicates that the segment copying to remote storage is started but not yet finished.
+     * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} - This state indicates that the segment copying to remote storage is finished.
+     * <br>
+     * The leader broker copies the log segments to the remote storage and puts the remote log segment metadata with the
+     * state as “COPY_SEGMENT_STARTED” and updates the state as “COPY_SEGMENT_FINISHED” once the copy is successful.
+     * <p></p>
+     * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED} - This state indicates that the segment deletion is started but not yet finished.
+     * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED} - This state indicates that the segment is deleted successfully.
+     * <br>
+     * Leader partitions publish both the above delete segment events when remote log retention is reached for the
+     * respective segments. Remote Partition Removers also publish these events when a segment is deleted as part of
+     * the remote partition deletion.
+     *
+     * @param remoteLogSegmentMetadataUpdate update of the remote log segment metadata.
+     * @throws RemoteStorageException          if there are any storage related errors occurred.
+     * @throws RemoteResourceNotFoundException when there are no resources associated with the given remoteLogSegmentMetadataUpdate.
+     */
+    void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate remoteLogSegmentMetadataUpdate)
+            throws RemoteStorageException;
+
+    /**
+     * Returns {@link RemoteLogSegmentMetadata} if it exists for the given topic partition containing the offset with
+     * the given leader-epoch for the offset, else returns {@link Optional#empty()}.
+     *
+     * @param topicIdPartition topic partition
+     * @param offset           offset
+     * @param epochForOffset   leader epoch for the given offset
+     * @return the requested remote log segment metadata if it exists.
+     * @throws RemoteStorageException if there are any storage related errors occurred.
+     */
+    Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition,
+                                                                long offset,
+                                                                int epochForOffset)
+            throws RemoteStorageException;
+
+    /**
+     * Returns the highest log offset of topic partition for the given leader epoch in remote storage. This is used by
+     * remote log management subsystem to know upto which offset the segments have been copied to remote storage for

Review comment:
       upto => up tp




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