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/09/14 12:21:10 UTC

[kafka] branch trunk updated: KAFKA-15442: add a section in doc for tiered storage (#14382)

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new ac39342d470 KAFKA-15442: add a section in doc for tiered storage  (#14382)
ac39342d470 is described below

commit ac39342d47062c1a9aec9b744b40ad7fd2de27b2
Author: Luke Chen <sh...@gmail.com>
AuthorDate: Thu Sep 14 20:21:00 2023 +0800

    KAFKA-15442: add a section in doc for tiered storage  (#14382)
    
    Added 6.11: Tiered Storage section and notable changes ini v3.6.0
    
    Reviewers: Satish Duggana <sa...@apache.org>, Gantigmaa Selenge <gs...@redhat.com>
---
 docs/ops.html     | 92 +++++++++++++++++++++++++++++++++++++++++++++++++++++++
 docs/toc.html     |  8 +++++
 docs/upgrade.html |  5 +++
 3 files changed, 105 insertions(+)

diff --git a/docs/ops.html b/docs/ops.html
index 7f56c8567d0..7c4b85aacd1 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -3859,6 +3859,98 @@ listeners=CONTROLLER://:9093
 
 # Other configs ...</pre>
 
+
+<h3 class="anchor-heading"><a id="tiered_storage" class="anchor-link"></a><a href="#kraft">6.11 Tiered Storage</a></h3>
+
+<h4 class="anchor-heading"><a id="tiered_storage_overview" class="anchor-link"></a><a href="#tiered_storage_overview">Tiered Storage Overview</a></h4>
+
+<p>Kafka data is mostly consumed in a streaming fashion using tail reads. Tail reads leverage OS's page cache to serve the data instead of disk reads.
+  Older data is typically read from the disk for backfill or failure recovery purposes and is infrequent.</p>
+
+<p>In the tiered storage approach, Kafka cluster is configured with two tiers of storage - local and remote.
+  The local tier is the same as the current Kafka that uses the local disks on the Kafka brokers to store the log segments.
+  The new remote tier uses external storage systems, such as HDFS or S3, to store the completed log segments.
+  Please check <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage">KIP-405</a> for more information.
+</p>
+
+<p><b>Note: Tiered storage is considered as an early access feature, and is not recommended for use in production environments</b></p>
+
+<h4 class="anchor-heading"><a id="tiered_storage_config" class="anchor-link"></a><a href="#tiered_storage_config">Configuration</a></h4>
+
+<h5 class="anchor-heading"><a id="tiered_storage_config_broker" class="anchor-link"></a><a href="#tiered_storage_config_broker">Broker Configurations</a></h5>
+
+<p>By default, Kafka server will not enable tiered storage feature. <code>remote.log.storage.system.enable</code>
+  is the property to control whether to enable tiered storage functionality in a broker or not. Setting it to "true" enables this feature.
+</p>
+
+<p><code>RemoteStorageManager</code> is an interface to provide the lifecycle of remote log segments and indexes. Kafka server
+  doesn't provide out-of-the-box implementation of RemoteStorageManager. Configuring <code>remote.log.storage.manager.class.name</code>
+  and <code>remote.log.storage.manager.class.path</code> to specify the implementation of RemoteStorageManager.
+</p>
+
+<p><code>RemoteLogMetadataManager</code> is an interface to provide the lifecycle of metadata about remote log segments with strongly consistent semantics.
+  By default, Kafka provides an implementation with storage as an internal topic. This implementation can be changed by configuring
+  <code>remote.log.metadata.manager.class.name</code> and <code>remote.log.metadata.manager.class.path</code>.
+  When adopting the default kafka internal topic based implementation, <code>remote.log.metadata.manager.listener.name</code>
+  is a mandatory property to specify which listener the clients created by the default RemoteLogMetadataManager implementation.
+</p>
+
+
+<h5 class="anchor-heading"><a id="tiered_storage_config_topic" class="anchor-link"></a><a href="#tiered_storage_config_topic">Topic Configurations</a></h5>
+
+<p>After correctly configuring broker side configurations for tiered storage feature, there are still configurations in topic level needed to be set.
+  <code>remote.storage.enable</code> is the switch to determine if a topic wants to use tiered storage or not. By default it is set to false.
+  After enabling <code>remote.storage.enable</code> property, the next thing to consider is the log retention.
+  When tiered storage is enabled for a topic, there are 2 additional log retention configurations to set:
+
+<ul>
+  <li><code>local.retention.ms</code></li>
+  <li><code>retention.ms</code></li>
+  <li><code>local.retention.bytes</code></li>
+  <li><code>retention.bytes</code></li>
+</ul>
+
+  The configuration prefixed with <code>local</code> are to specify the time/size the "local" log file can accept before moving to remote storage, and then get deleted.
+  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>
+
+<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.
+remote.log.metadata.manager.listener.name=PLAINTEXT
+</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
+</pre>
+</p>
+
+<p>Then, after the active segment is rolled, the old segment should be moved to the remote storage and get deleted.
+</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:
+<ul>
+  <li>No support for clusters with multiple log directories (i.e. JBOD feature)</li>
+  <li>No support for compacted topics</li>
+  <li>Cannot disable tiered storage at the topic level</li>
+  <li>Deleting tiered storage enabled topics is required before disabling tiered storage at the broker level</li>
+  <li>Admin actions related to tiered storage feature are only supported on clients from version 3.0 onwards</li>
+</ul>
+</p>
+
+<p>For more information, please check <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+Early+Access+Release+Notes">Tiered Storage Early Access Release Note</a>.
+</p>
+
+
 </script>
 
 <div class="p-ops"></div>
diff --git a/docs/toc.html b/docs/toc.html
index 88dd62c92dd..737ef887cd1 100644
--- a/docs/toc.html
+++ b/docs/toc.html
@@ -169,6 +169,14 @@
                         <li><a href="#kraft_zk_migration">ZooKeeper to KRaft Migration</a></li>
                     </ul>
                 </li>
+                <li><a href="#tiered_storage">6.11 Tiered Storage</a>
+                    <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_limitation">Limitations</a></li>
+                    </ul>
+                </li>
             </ul>
         </li>
         <li><a href="#security">7. Security</a>
diff --git a/docs/upgrade.html b/docs/upgrade.html
index 13e76b79cc3..ca86b1c839f 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -50,6 +50,11 @@
             <code>replication.policy.internal.topic.separator.enabled</code>
             property. If upgrading from 3.0.x or earlier, it may be necessary to set this property to <code>false</code>; see the property's
             <a href="#mirror_connector_replication.policy.internal.topic.separator.enabled">documentation</a> for more details.</li>
+        <li>Early access of tiered storage feature is available, and it is not recommended for use in production environments.
+            Welcome to test it and provide any feedback to us.
+            For more information about the early access tiered storage feature, please check <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage">KIP-405</a> and
+            <a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+Early+Access+Release+Notes">Tiered Storage Early Access Release Note</a>.
+        </li>
     </ul>
 
 <h4><a id="upgrade_3_5_0" href="#upgrade_3_5_0">Upgrading to 3.5.0 from any version 0.8.x through 3.4.x</a></h4>