Interface RemoteStorageManager

All Superinterfaces:
AutoCloseable, Closeable, Configurable

@Evolving public interface RemoteStorageManager extends Configurable, Closeable
This interface provides the lifecycle of remote log segments that includes copy, fetch, and delete from remote storage.

Each upload or copy of a segment is initiated with RemoteLogSegmentMetadata containing RemoteLogSegmentId which is universally unique even for the same topic partition and offsets.

RemoteLogSegmentMetadata is stored in RemoteLogMetadataManager before and after copy/delete operations on RemoteStorageManager with the respective RemoteLogSegmentState. RemoteLogMetadataManager is responsible for storing and fetching metadata about the remote log segments in a strongly consistent manner. This allows RemoteStorageManager to have eventual consistency on metadata (although the data is stored in strongly consistent semantics).

All properties prefixed with the config: "remote.log.storage.manager.impl.prefix" (default value is "rsm.config.") are passed when Configurable.configure(Map) is invoked on this instance.

  • Method Details

    • copyLogSegmentData

      Optional<RemoteLogSegmentMetadata.CustomMetadata> copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata, LogSegmentData logSegmentData) throws RemoteStorageException
      Copies the given LogSegmentData provided for the given remoteLogSegmentMetadata. This includes log segment and its auxiliary indexes like offset index, time index, transaction index, leader epoch index, and producer snapshot index.

      Invoker of this API should always send a unique id as part of RemoteLogSegmentMetadata.remoteLogSegmentId() even when it retries to invoke this method for the same log segment data.

      This operation is expected to be idempotent. If a copy operation is retried and there is existing content already written, it should be overwritten, and do not throw RemoteStorageException

      Parameters:
      remoteLogSegmentMetadata - metadata about the remote log segment.
      logSegmentData - data to be copied to tiered storage.
      Returns:
      custom metadata to be added to the segment metadata after copying.
      Throws:
      RemoteStorageException - if there are any errors in storing the data of the segment.
    • fetchLogSegment

      InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition) throws RemoteStorageException
      Returns the remote log segment data file/object as InputStream for the given RemoteLogSegmentMetadata starting from the given startPosition. The stream will end at the end of the remote log segment data file/object.
      Parameters:
      remoteLogSegmentMetadata - metadata about the remote log segment.
      startPosition - start position of log segment to be read, inclusive.
      Returns:
      input stream of the requested log segment data.
      Throws:
      RemoteStorageException - if there are any errors while fetching the desired segment.
      RemoteResourceNotFoundException - the requested log segment is not found in the remote storage.
    • fetchLogSegment

      InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition, int endPosition) throws RemoteStorageException
      Returns the remote log segment data file/object as InputStream for the given 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.
      Parameters:
      remoteLogSegmentMetadata - metadata about the remote log segment.
      startPosition - start position of log segment to be read, inclusive.
      endPosition - end position of log segment to be read, inclusive.
      Returns:
      input stream of the requested log segment data.
      Throws:
      RemoteStorageException - if there are any errors while fetching the desired segment.
      RemoteResourceNotFoundException - the requested log segment is not found in the remote storage.
    • fetchIndex

      InputStream fetchIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata, RemoteStorageManager.IndexType indexType) throws RemoteStorageException
      Returns the index for the respective log segment of RemoteLogSegmentMetadata.

      Note: The transaction index may not exist because of no transactional records. In this case, it should throw a RemoteResourceNotFoundException, instead of returning null.

      Parameters:
      remoteLogSegmentMetadata - metadata about the remote log segment.
      indexType - type of the index to be fetched for the segment.
      Returns:
      input stream of the requested index.
      Throws:
      RemoteStorageException - if there are any errors while fetching the index.
      RemoteResourceNotFoundException - the requested index is not found in the remote storage
    • deleteLogSegmentData

      void deleteLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException
      Deletes the resources associated with the given remoteLogSegmentMetadata. Deletion is considered as successful if this call returns successfully without any errors. It will throw RemoteStorageException if there are any errors in deleting the file.

      This operation is expected to be idempotent. If resources are not found, it is not expected to throw RemoteResourceNotFoundException as it may be already removed from a previous attempt.

      Parameters:
      remoteLogSegmentMetadata - metadata about the remote log segment to be deleted.
      Throws:
      RemoteStorageException - if there are any storage related errors occurred.