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



##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 {
+
+    /**
+     * Adds {@link RemoteLogSegmentMetadata} with the containing {@link 
RemoteLogSegmentId} into {@link RemoteLogMetadataManager}.
+     * <p>
+     * RemoteLogSegmentMetadata is identified by RemoteLogSegmentId and it 
should have the initial state which is {@link 
RemoteLogSegmentState#COPY_SEGMENT_STARTED}.
+     * <p>
+     * {@link #updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate)} 
should be used to update an existing RemoteLogSegmentMetadata.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @throws RemoteStorageException   if there are any storage related 
errors occurred.
+     * @throws IllegalArgumentException if the given metadata instance does 
not have the state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}
+     */
+    void addRemoteLogSegmentMetadata(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.
+     * @throws IllegalArgumentException        if the given metadata instance 
has the state as {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}
+     */
+    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 up to 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>
+     * In the case of default RLMM implementation, remote partition remover 
processes {@link RemotePartitionDeleteState#DELETE_PARTITION_MARKED}

Review comment:
       Should we remove references to implementation from the interface? We can 
keep this doc in the implementation class.

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * 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 {
+
+    /**
+     * 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 within 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 (inclusive).
+     * @param endOffset           End offset of this segment (inclusive).
+     * @param maxTimestamp        Maximum timestamp in this segment.
+     * @param brokerId            Broker id from which this event is generated.
+     * @param eventTimestamp      Epoch time in milli seconds 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 within 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 
TreeMap<>(segmentLeaderEpochs));
+    }
+
+    /**
+     * Creates an instance with the given metadata of remote log segment and 
its 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 (inclusive).
+     * @param endOffset           End offset of this segment (inclusive).
+     * @param maxTimestamp        Maximum timestamp in this segment
+     * @param brokerId            Broker id from which this event is generated.
+     * @param eventTimestamp      Epoch time in milli seconds 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 within this segment
+     */
+    public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId,
+                                    long startOffset,
+                                    long endOffset,
+                                    long maxTimestamp,
+                                    int brokerId,
+                                    long eventTimestamp,
+                                    int segmentSizeInBytes,
+                                    Map<Integer, Long> segmentLeaderEpochs) {
+        this(remoteLogSegmentId,
+                startOffset,
+                endOffset,
+                maxTimestamp,
+                brokerId,
+                eventTimestamp, segmentSizeInBytes,
+                RemoteLogSegmentState.COPY_SEGMENT_STARTED,
+                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 within this segment.
+     */
+    public long maxTimestamp() {
+        return maxTimestamp;
+    }
+
+    /**
+     * @return Map of leader epoch vs offset for the records available in this 
segment.
+     */
+    public NavigableMap<Integer, Long> segmentLeaderEpochs() {
+        return segmentLeaderEpochs;
+    }
+
+    /**
+     * @return Broker id from which this event is generated.
+     */
+    public int brokerId() {
+        return brokerId;
+    }
+
+    /**
+     * Returns the current state of this remote log segment. It can be any of 
the below
+     * <ul>
+     *     {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}
+     *     {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}
+     *     {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}
+     *     {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}
+     * </ul>
+     */
+    public RemoteLogSegmentState state() {
+        return state;
+    }
+
+    /**
+     * Creates a new RemoteLogSegmentMetadata applying the given {@code 
rlsmUpdate} on this instance. This method will
+     * not update this instance.
+     *
+     * @param rlsmUpdate update to be applied.
+     * @return a new instance created by applying the given update on this 
instance.
+     */
+    public RemoteLogSegmentMetadata 
createRemoteLogSegmentWithUpdates(RemoteLogSegmentMetadataUpdate rlsmUpdate) {

Review comment:
       Should we call this method just as `createWithUpdates`?

##########
File path: clients/src/main/java/org/apache/kafka/common/TopicIdPartition.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.util.Objects;
+
+/**
+ * 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 {
+
+    private final Uuid topicId;
+    private final TopicPartition topicPartition;
+
+    public TopicIdPartition(Uuid topicId, TopicPartition topicPartition) {
+        this.topicId = Objects.requireNonNull(topicId, "topicId can not be 
null");
+        this.topicPartition = Objects.requireNonNull(topicPartition, 
"topicPartition can not be null");
+    }
+
+    /**
+     * @return Universally unique id representing this topic partition.
+     */
+    public Uuid topicId() {
+        return topicId;
+    }
+
+    /**
+     * @return Topic partition representing this instance.
+     */
+    public TopicPartition topicPartition() {

Review comment:
       Should we call this just `partition()` ?

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * 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 {
+
+    /**
+     * 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;

Review comment:
       Could we include the time unit in the variable name?

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.util.Objects;
+
+/**
+ * This class represents the metadata about the remote partition. It can be 
updated with {@link 
RemoteLogMetadataManager#putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata)}.
+ * Possible state transitions are mentioned at {@link 
RemotePartitionDeleteState}.
+ */
+@InterfaceStability.Evolving
+public class RemotePartitionDeleteMetadata {
+
+    private final TopicIdPartition topicIdPartition;
+    private final RemotePartitionDeleteState state;
+    private final long eventTimestamp;

Review comment:
       Could we include the time unit in the variable name?

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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 have eventual consistency on 
metadata (although the data is stored
+ * in strongly consistent semantics).
+ */
+@InterfaceStability.Evolving
+public interface RemoteStorageManager extends Configurable, Closeable {
+
+    /**
+     * Type of the index file.
+     */
+    enum IndexType {
+        /**
+         * Represents offset index.
+         */
+        Offset,
+
+        /**
+         * Represents timestamp index.
+         */
+        Timestamp,
+
+        /**
+         * Represents producer snapshot index.
+         */
+        ProducerSnapshot,
+
+        /**
+         * Represents transaction index.
+         */
+        Transaction,
+
+        /**
+         * Represents leader epoch index.
+         */
+        LeaderEpoch,
+    }
+
+    /**
+     * Copies the given {@link LogSegmentData} provided for the given {@code 
remoteLogSegmentMetadata}. This includes
+     * log segment and its auxiliary indexes like offset index, time index, 
transaction index, leader epoch index, and
+     * producer snapshot index.
+     * <p>
+     * Invoker of this API should always send a unique id as part of {@link 
RemoteLogSegmentMetadata#remoteLogSegmentId()}
+     * even when it retries to invoke this method for the same log segment 
data.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @param logSegmentData           data to be copied to tiered storage.
+     * @throws RemoteStorageException if there are any errors in storing the 
data of the segment.
+     */
+    void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                            LogSegmentData logSegmentData)
+            throws RemoteStorageException;
+
+    /**
+     * Returns the remote log segment data file/object as InputStream for the 
given {@link RemoteLogSegmentMetadata}
+     * starting from the given startPosition. The stream will end at the end 
of the remote log segment data file/object.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @param startPosition            start position of log segment to be 
read, inclusive.
+     * @return input stream of the requested log segment data.
+     * @throws RemoteStorageException          if there are any errors while 
fetching the desired segment.
+     * @throws RemoteResourceNotFoundException when there are no resources 
associated with the given remoteLogSegmentMetadata.
+     */
+    InputStream fetchLogSegment(RemoteLogSegmentMetadata 
remoteLogSegmentMetadata,
+                                int startPosition) throws 
RemoteStorageException;
+
+    /**
+     * Returns the remote log segment data file/object as InputStream for the 
given {@link RemoteLogSegmentMetadata}
+     * starting from the given startPosition. The stream will end at the 
smaller of endPosition and the end of the
+     * remote log segment data file/object.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @param startPosition            start position of log segment to be 
read, inclusive.
+     * @param endPosition              end position of log segment to be read, 
inclusive.
+     * @return input stream of the requested log segment data.
+     * @throws RemoteStorageException          if there are any errors while 
fetching the desired segment.
+     * @throws RemoteResourceNotFoundException when there are no resources 
associated with the given remoteLogSegmentMetadata.
+     */
+    InputStream fetchLogSegment(RemoteLogSegmentMetadata 
remoteLogSegmentMetadata,
+                                int startPosition,
+                                int endPosition) throws RemoteStorageException;
+
+    /**
+     * Returns the index for the respective log segment of {@link 
RemoteLogSegmentMetadata}.
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment.
+     * @param indexType                type of the index to be fetched for the 
segment.
+     * @return input stream of the requested index.
+     * @throws RemoteStorageException          if there are any errors while 
fetching the index.
+     * @throws RemoteResourceNotFoundException when there are no resources 
associated with the given remoteLogSegmentMetadata.
+     */
+    InputStream fetchIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                           IndexType indexType) throws RemoteStorageException;
+
+    /**
+     * Deletes the resources associated with the given {@code 
remoteLogSegmentMetadata}. Deletion is considered as
+     * successful if this call returns successfully without any errors. It 
will throw {@link RemoteStorageException} if
+     * there are any errors in deleting the file.
+     * <p>
+     *
+     * @param remoteLogSegmentMetadata metadata about the remote log segment 
to be deleted.
+     * @throws RemoteResourceNotFoundException if the requested resource is 
not found
+     * @throws RemoteStorageException          if there are any storage 
related errors occurred.
+     * @throws RemoteResourceNotFoundException when there are no resources 
associated with the given remoteLogSegmentMetadata.
+     */
+    void deleteLogSegmentData(RemoteLogSegmentMetadata 
remoteLogSegmentMetadata) throws RemoteStorageException;
+

Review comment:
       nit: extra line can be removed

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java
##########
@@ -0,0 +1,282 @@
+/*
+ * 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.util.Collections;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * 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 {
+
+    /**
+     * 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;

Review comment:
       Same comment as above - time unit in the variable name.

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.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 {
+
+    /**
+     * Universally unique remote log segment id.
+     */
+    private final RemoteLogSegmentId remoteLogSegmentId;
+
+    /**
+     * Epoch time at which this event is generated.
+     */
+    private final long eventTimestamp;

Review comment:
       Could we include the time unit in the variable name?

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.File;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * This represents all the required data and indexes for a specific log 
segment that needs to be stored in the remote
+ * storage. This is passed with {@link 
RemoteStorageManager#copyLogSegmentData(RemoteLogSegmentMetadata, 
LogSegmentData)}
+ * while copying a specific log segment to the remote storage.
+ */
+@InterfaceStability.Evolving
+public class LogSegmentData {
+
+    private final File logSegment;
+    private final File offsetIndex;
+    private final File timeIndex;
+    private final File txnIndex;
+    private final File producerSnapshotIndex;
+    private final ByteBuffer leaderEpochIndex;
+
+    /**
+     * Creates a LogSegmentData instance with data and indexes.
+     *
+     * @param logSegment            actual log segment file
+     * @param offsetIndex           offset index file
+     * @param timeIndex             time index file
+     * @param txnIndex              transaction index file
+     * @param producerSnapshotIndex producer snapshot until this segment
+     * @param leaderEpochIndex      leader-epoch-index until this segment
+     */
+    public LogSegmentData(File logSegment,
+                          File offsetIndex,
+                          File timeIndex,
+                          File txnIndex,
+                          File producerSnapshotIndex,
+                          ByteBuffer leaderEpochIndex) {
+        this.logSegment = Objects.requireNonNull(logSegment, "logSegment can 
not be null");
+        this.offsetIndex = Objects.requireNonNull(offsetIndex, "offsetIndex 
can not be null");
+        this.timeIndex = Objects.requireNonNull(timeIndex, "timeIndex can not 
be null");
+        this.txnIndex = Objects.requireNonNull(txnIndex, "txnIndex can not be 
null");
+        this.producerSnapshotIndex = 
Objects.requireNonNull(producerSnapshotIndex, "producerSnapshotIndex can not be 
null");
+        this.leaderEpochIndex = Objects.requireNonNull(leaderEpochIndex, 
"leaderEpochIndex can not be null");
+    }
+
+    /**
+     * @return Log segment file of this segment.
+     */
+    public File logSegment() {
+        return logSegment;
+    }
+
+    /**
+     * @return Offset index file.
+     */
+    public File offsetIndex() {
+        return offsetIndex;
+    }
+
+    /**
+     * @return Time index file of this segment.
+     */
+    public File timeIndex() {
+        return timeIndex;
+    }
+
+    /**
+     * @return Transaction index file of this segment.
+     */
+    public File txnIndex() {
+        return txnIndex;
+    }
+
+    /**
+     * @return Producer snapshot file until this segment.
+     */
+    public File producerSnapshotIndex() {
+        return producerSnapshotIndex;
+    }
+
+    /**
+     * @return Leader epoch index until this segment.
+     */
+    public ByteBuffer leaderEpochIndex() {
+        return leaderEpochIndex;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        LogSegmentData that = (LogSegmentData) o;
+        return Objects.equals(logSegment, that.logSegment) && Objects
+                .equals(offsetIndex, that.offsetIndex) && Objects
+                       .equals(timeIndex, that.timeIndex) && Objects

Review comment:
       The indentation looks a bit off here.

##########
File path: 
clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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 have eventual consistency on 
metadata (although the data is stored
+ * in strongly consistent semantics).
+ */
+@InterfaceStability.Evolving
+public interface RemoteStorageManager extends Configurable, Closeable {
+
+    /**
+     * Type of the index file.
+     */
+    enum IndexType {
+        /**
+         * Represents offset index.
+         */
+        Offset,

Review comment:
       Can we change the naming to use `ALL_CAPS`?
   For example, compare with the definition of `RemotePartitionDeleteState` in 
this PR.




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


Reply via email to