Versions Compared

Key

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

...

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: -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. Common client propoerties can be configured with `remote.log.metadata.common.client.` prefix.  User can also pass properties specific to producer/consumer with `remote.log.metadata.producer.` and `remote.log.metadata.consumer.` prefixes. These will override properties with `remote.log.metadata.common.client.` prefix.

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 )

...

Follow the steps mentioned in Kafka upgrade to reach the state where all brokers are running on the latest binaries with the respective "inter.broker.protocol" and "log.message.format" versions. Tiered storage requires the message format to be > 0.11.

...

You can enable tiered storage by setting “remote.log.storage.enable” to true on the desired topics. Before enabling tiered storage, you should make sure the producer snapshots are built for all the segments for that topic in all followers. You should wait till the log retention occurs for all the segments so that all the segments have producer snapshots. Because follower replicas for topics with tier storage enabled, need the respective producer snapshot for each segment for reconciliating reconciling the state as mentioned in the earlier follower fetch protocol section.

Downgrade

Downgrade to earlier versions(> 2.1) is possible but the data available only on remote storage will not be available. There will be a few files that are created in remote index cache directory($log.dir/remote-log-index-cache) and other remote log segment metadata cache files that need to be cleaned up by the user. We may provide a script to cleanup the cache files created by tiered storage.Users have to manually delete the data in remote storage based on the bucket or dir configured with tiered storage.

Limitations

  • Once tier storage is enabled for a topic, it can not be disabled. We will add this feature in future versions. One possible workaround is to create a new topic and copy the data from the desired offset and delete the old topic. 
  • Multiple Log dirs on a broker are not supported (JBOD related features).
  • Tiered storage is not supported for compacted topics.

Integration and System tests

...

  • Enhance RLMM local file-based cache with RocksDB to avoid loading the whole cache inmemory. 
  • Enhance RLMM implementation based on topic based storage pointing to a target Kafka cluster instead of using as a system level topic within the cluster.
  • Improve default RLMM implementation with a less chatty protocol.
  • Support disabling tiered storage for a topic. 
  • Add a system level config to enable tiered storage for all the topics in a cluster.
  • Recovery mechanism incase of the broker or cluster failure.
    • This is to be done by fetching the remote log metadata from RemoteStorageManager.
  • Recovering  from remote log metadata topic partitions truncation
  • Extract RPMM as a separate task and allow any RLMM implementation to reuse the task for deletion of remote segments and complete the remote partition deletion.

...