You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by sh...@apache.org on 2023/10/18 09:35:23 UTC

[kafka-site] branch asf-site updated: MINOR: add docs for tiered storage quick start (#562)

This is an automated email from the ASF dual-hosted git repository.

showuon pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
     new 81773f6d MINOR: add docs for tiered storage quick start (#562)
81773f6d is described below

commit 81773f6d1afe2ce8e6305299a085f8dd559d8140
Author: Luke Chen <sh...@gmail.com>
AuthorDate: Wed Oct 18 17:35:18 2023 +0800

    MINOR: add docs for tiered storage quick start (#562)
    
    Updated 3.6 docs from kafka repo: tiered storage quick start guide: apache/kafka#14528
    
    Reviewers:  Divij Vaidya <di...@amazon.com>, Kamal Chandraprakash <ka...@gmail.com>
---
 36/ops.html | 82 +++++++++++++++++++++++++++++++++++++++++++++++++++++++------
 36/toc.html |  2 +-
 2 files changed, 76 insertions(+), 8 deletions(-)

diff --git a/36/ops.html b/36/ops.html
index 3ee9cac4..4414a0b8 100644
--- a/36/ops.html
+++ b/36/ops.html
@@ -3984,27 +3984,95 @@ listeners=CONTROLLER://:9093
   If unset, The value in <code>retention.ms</code> and <code>retention.bytes</code> will be used.
 </p>
 
-<h4 class="anchor-heading"><a id="tiered_storage_config_ex" class="anchor-link"></a><a href="#tiered_storage_config_ex">Configurations Example</a></h4>
+<h4 class="anchor-heading"><a id="tiered_storage_config_ex" class="anchor-link"></a><a href="#tiered_storage_config_ex">Quick Start Example</a></h4>
+
+<p>Apache Kafka doesn't provide an out-of-the-box RemoteStorageManager implementation. To have a preview of the tiered storage
+  feature, the <a href="https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java">LocalTieredStorage</a>
+  implemented for integration test can be used, which will create a temporary directory in local storage to simulate the remote storage.
+</p>
+
+<p>To adopt the `LocalTieredStorage`, the test library needs to be built locally</p>
+<pre># please checkout to the specific version tag you're using before building it
+# ex: `git checkout 3.6.0`
+./gradlew clean :storage:testJar</pre>
+<p>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.</p>
 
-<p>Here is a sample configuration to enable tiered storage feature in broker side:
 <pre>
 # 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
 </pre>
 </p>
 
-<p>After broker is started, creating a topic with tiered storage enabled, and a small log time retention value to try this feature:
-<pre>bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:9092 --config remote.storage.enable=true --config local.retention.ms=1000
+<p>Following <a href="#quickstart_startserver">quick start guide</a> to start up the kafka environment.
+  Then, create a topic with tiered storage enabled with configs:
+
+<pre>
+# 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
 </pre>
 </p>
 
+<p>Try to send messages to the `tieredTopic` topic to roll the log segment:</p>
+
+<pre>
+bin/kafka-producer-perf-test.sh --topic tieredTopic --num-records 1200 --record-size 1024 --throughput -1 --producer-props bootstrap.servers=localhost:9092
+</pre>
+
 <p>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:
 </p>
 
+<pre> > 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
+</pre>
+
+<p>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.</p>
+
+<pre>bin/kafka-console-consumer.sh --topic tieredTopic --from-beginning --max-messages 1 --bootstrap-server localhost:9092 --property print.offset=true</pre>
+
+<p>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.</p>
+
+<pre>bin/kafka-topics.sh --delete --topic tieredTopic --bootstrap-server localhost:9092</pre>
+
+<p>After topics are deleted, you're safe to set <code>remote.log.storage.system.enable=false</code> in the broker configuration.</p>
+
 <h4 class="anchor-heading"><a id="tiered_storage_limitation" class="anchor-link"></a><a href="#tiered_storage_limitation">Limitations</a></h4>
 
 <p>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/36/toc.html b/36/toc.html
index 737ef887..73bd66ae 100644
--- a/36/toc.html
+++ b/36/toc.html
@@ -173,7 +173,7 @@
                     <ul>
                         <li><a href="#tiered_storage_overview">Tiered Storage Overview</a></li>
                         <li><a href="#tiered_storage_config">Configuration</a></li>
-                        <li><a href="#tiered_storage_config_ex">Configurations Example</a></li>
+                        <li><a href="#tiered_storage_config_ex">Quick Start Example</a></li>
                         <li><a href="#tiered_storage_limitation">Limitations</a></li>
                     </ul>
                 </li>