Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

  • receives callback events for leadership changes and stop/delete events of topic partitions on a broker.
  • delegates copy, read, and delete of topic partition segments to a pluggable storage manager(viz RemoteStorageManager) implementation and maintains respective remote log segment metadata through RemoteLogMetadataManager.

`RemoteLogManager` is an internal component and it is not a public API. 

`RemoteStorageManager` is an interface to provide the lifecycle of remote log segments and indexes. More details about how we arrived at this interface are discussed in the document. We will provide a simple implementation of RSM to get a better understanding of the APIs. HDFS and S3 implementation are planned to be hosted in external repos and these will not be part of Apache Kafka repo. This is inline with the approach taken for Kafka connectors.

...

Metadata of remote log segments are stored in an internal non compact topic called `__remote_log_metadata`. This topic can be created with default partitions count as 50. Users can configure the partitions count and replication factor etc as mentioned in the config section.

...

remote.log.metadata.topic.replication.factor


Replication factor of the topic

Default: 3

remote.log.metadata.topic.partitions

No of partitions of the topic

Default: 50

remote.log.metadata.topic.retention.ms

Retention of the topic in milli seconds

Default: 365 * 24 * 60 * 60 * 1000  (1 yr) -1, that means unlimited. 

Users can configure this value to based on their usecases. To avoid any data loss, this value should be more than the maximum retention period of any topic enaled with tiered storage in the cluster. 

remote.log.metadata.manager.listener.name

Listener name to be  used to connect to the local broker by RemoteLogMetadataManager implementation on the broker. Respective endpoint address is passed with  "bootstrap.servers" property while invoking RemoteLogMetadataManager#configure(Map<String, ?> props). 

This is used by kafka clients created in RemoteLogMetadataManager implementation.

remote.log.metadata.*

Default RLMM implementation creates producer and consumer instances. User can pass properties specific to producer/consumer with `remote.log.metadata.producer.` and `remote.log.metadata.consumer.` perfixes. 

Any other properties should be prefixed with "remote.log.metadata." and these will be passed to RemoteLogMetadataManager#configure(Map<String, ?> props).

For ex: Security configuration to connect to the local broker for the listener name configured are passed with props.

remote.partition.remover.task.interval.msThe interval at which remote partition remover runs to delete the remote storage of the partitions marked for deletion.
Default value: 3600000 (1 hr )

...