Interface RemoteStorageManager
- All Superinterfaces:
AutoCloseable
,Closeable
,Configurable
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.
-
Nested Class Summary
Modifier and TypeInterfaceDescriptionstatic enum
Type of the index file. -
Method Summary
Modifier and TypeMethodDescriptioncopyLogSegmentData
(RemoteLogSegmentMetadata remoteLogSegmentMetadata, LogSegmentData logSegmentData) Copies the givenLogSegmentData
provided for the givenremoteLogSegmentMetadata
.void
deleteLogSegmentData
(RemoteLogSegmentMetadata remoteLogSegmentMetadata) Deletes the resources associated with the givenremoteLogSegmentMetadata
.fetchIndex
(RemoteLogSegmentMetadata remoteLogSegmentMetadata, RemoteStorageManager.IndexType indexType) Returns the index for the respective log segment ofRemoteLogSegmentMetadata
.fetchLogSegment
(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition) Returns the remote log segment data file/object as InputStream for the givenRemoteLogSegmentMetadata
starting from the given startPosition.fetchLogSegment
(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition, int endPosition) Returns the remote log segment data file/object as InputStream for the givenRemoteLogSegmentMetadata
starting from the given startPosition.Methods inherited from interface org.apache.kafka.common.Configurable
configure
-
Method Details
-
copyLogSegmentData
Optional<RemoteLogSegmentMetadata.CustomMetadata> copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata, LogSegmentData logSegmentData) throws RemoteStorageException Copies the givenLogSegmentData
provided for the givenremoteLogSegmentMetadata
. 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 givenRemoteLogSegmentMetadata
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 givenRemoteLogSegmentMetadata
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 ofRemoteLogSegmentMetadata
.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 givenremoteLogSegmentMetadata
. Deletion is considered as successful if this call returns successfully without any errors. It will throwRemoteStorageException
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.
-