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/10 20:24:25 UTC

[GitHub] [kafka] kowshik 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.

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