diff --git a/docs/ops.html b/docs/ops.html index 3ee9cac4237..4414a0b86ad 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3984,27 +3984,95 @@ listeners=CONTROLLER://:9093 If unset, The value in retention.ms and retention.bytes will be used.

-

Configurations Example

+

Quick Start Example

+ +

Apache Kafka doesn't provide an out-of-the-box RemoteStorageManager implementation. To have a preview of the tiered storage + feature, the LocalTieredStorage + implemented for integration test can be used, which will create a temporary directory in local storage to simulate the remote storage. +

+ +

To adopt the `LocalTieredStorage`, the test library needs to be built locally

+
# please checkout to the specific version tag you're using before building it
+# ex: `git checkout 3.6.0`
+./gradlew clean :storage:testJar
+

After build successfully, there should be a `kafka-storage-x.x.x-test.jar` file under `storage/build/libs`. +Next, setting configurations in the broker side to enable tiered storage feature.

-

Here is a sample configuration to enable tiered storage feature in broker side:

 # Sample Zookeeper/Kraft broker server.properties listening on PLAINTEXT://:9092
 remote.log.storage.system.enable=true
-# Please provide the implementation for remoteStorageManager. This is the mandatory configuration for tiered storage.
-# remote.log.storage.manager.class.name=org.apache.kafka.server.log.remote.storage.NoOpRemoteStorageManager
-# Using the "PLAINTEXT" listener for the clients in RemoteLogMetadataManager to talk to the brokers.
+
+# Setting the listener for the clients in RemoteLogMetadataManager to talk to the brokers.
 remote.log.metadata.manager.listener.name=PLAINTEXT
+
+# Please provide the implementation info for remoteStorageManager.
+# This is the mandatory configuration for tiered storage.
+# Here, we use the `LocalTieredStorage` built above.
+remote.log.storage.manager.class.name=org.apache.kafka.server.log.remote.storage.LocalTieredStorage
+remote.log.storage.manager.class.path=/PATH/TO/kafka-storage-x.x.x-test.jar
+
+# These 2 prefix are default values, but customizable
+remote.log.storage.manager.impl.prefix=rsm.config.
+remote.log.metadata.manager.impl.prefix=rlmm.config.
+
+# Configure the directory used for `LocalTieredStorage`
+# Note, please make sure the brokers need to have access to this directory
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if number of brokers in the cluster is more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
+
+# Try to speed up the log retention check interval for testing
+log.retention.check.interval.ms=1000
 

-

After broker is started, creating a topic with tiered storage enabled, and a small log time retention value to try this feature: -

bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:9092 --config remote.storage.enable=true --config local.retention.ms=1000
+

Following quick start guide to start up the kafka environment. + Then, create a topic with tiered storage enabled with configs: + +

+# remote.storage.enable=true -> enables tiered storage on the topic
+# local.retention.ms=1000 -> The number of milliseconds to keep the local log segment before it gets deleted.
+  Note that a local log segment is eligible for deletion only after it gets uploaded to remote.
+# retention.ms=3600000 -> when segments exceed this time, the segments in remote storage will be deleted
+# segment.bytes=1048576 -> for test only, to speed up the log segment rolling interval
+# file.delete.delay.ms=10000 -> for test only, to speed up the local-log segment file delete delay
+
+bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:9092 \
+--config remote.storage.enable=true --config local.retention.ms=1000 --config retention.ms=3600000 \
+--config segment.bytes=1048576 --config file.delete.delay.ms=1000
 

+

Try to send messages to the `tieredTopic` topic to roll the log segment:

+ +
+bin/kafka-producer-perf-test.sh --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1 --producer-props bootstrap.servers=localhost:9092
+
+

Then, after the active segment is rolled, the old segment should be moved to the remote storage and get deleted. + This can be verified by checking the remote log directory configured above. For example:

+
 > ls /tmp/kafka-remote-storage/kafka-tiered-storage/tieredTopic-0-jF8s79t9SrG_PNqlwv7bAA
+00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.index
+00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.snapshot
+00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.leader_epoch_checkpoint
+00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.timeindex
+00000000000000000000-knnxbs3FSRyKdPcSAOQC-w.log
+
+ +

Lastly, we can try to consume some data from the beginning and print offset number, to make sure it will successfully fetch offset 0 from the remote storage.

+ +
bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --property print.offset=true
+ +

Please note, if you want to disable tiered storage at the cluster level, you should delete the tiered storage enabled topics explicitly. + Attempting to disable tiered storage at the cluster level without deleting the topics using tiered storage will result in an exception during startup.

+ +
bin/kafka-topics.sh --delete --topic tieredTopic --bootstrap-server localhost:9092
+ +

After topics are deleted, you're safe to set remote.log.storage.system.enable=false in the broker configuration.

+

Limitations

While the early access release of Tiered Storage offers the opportunity to try out this new feature, it is important to be aware of the following limitations: diff --git a/docs/toc.html b/docs/toc.html index 737ef887cd1..73bd66ae41c 100644 --- a/docs/toc.html +++ b/docs/toc.html @@ -173,7 +173,7 @@