kamalcph commented on code in PR #14528:
URL: https://github.com/apache/kafka/pull/14528#discussion_r1356064302


##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar

Review Comment:
   The below configs are configured with default values but adding them to the 
quick-start page will clarity on rsm and rlmm properties as they can be 
custom-made:
   ```
   remote.log.storage.manager.impl.prefix=rsm.config.
   remote.log.metadata.manager.impl.prefix=rlmm.config.
   ```



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage
+# local.retention.ms=1000 -> the time before local segments are moving to 
remote storage (and then deleted)
+# retention.ms=300000 -> when segments exceed this time, the segments in 
remote storage will be deleted

Review Comment:
   Can we increase the `retention.ms` to 1 hour? The default is 7 days. User 
might gets confused why they are not seeing all the messages that were produced 
within 5 mins.



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage
+# local.retention.ms=1000 -> the time before local segments are moving to 
remote storage (and then deleted)
+# retention.ms=300000 -> when segments exceed this time, the segments in 
remote storage will be deleted
+# segment.bytes=1000000 -> for test only, to speed up the log segment rolling 
interval
+# file.delete.delay.ms=10000 -> for test only, to speed up the file delete 
delay

Review Comment:
   ```suggestion
   # file.delete.delay.ms=10000 -> for test only, to speed up the local-log 
segment file delete delay
   ```



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage
+# local.retention.ms=1000 -> the time before local segments are moving to 
remote storage (and then deleted)
+# retention.ms=300000 -> when segments exceed this time, the segments in 
remote storage will be deleted
+# segment.bytes=1000000 -> for test only, to speed up the log segment rolling 
interval
+# file.delete.delay.ms=10000 -> for test only, to speed up the 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=300000 --config 
segment.bytes=1000000 --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 1000 
--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>Last, we can try to consume some data in the offset 0, to make sure it will 
successfully fetch from the remote storage.</p>
+
+<pre>bin/kafka-console-consumer.sh --topic tieredTopic --partition 0 --offset 
0 --max-messages 1 --bootstrap-server localhost:9092</pre>

Review Comment:
   ```suggestion
   <pre>bin/kafka-console-consumer.sh --topic tieredTopic --partition 0 
--from-beginning --max-messages 1 --bootstrap-server localhost:9092 --property 
print.offset=true</pre>
   ```



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage

Review Comment:
   enabling tiered storage -> enables tiered storage on the topic



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage
+# local.retention.ms=1000 -> the time before local segments are moving to 
remote storage (and then deleted)

Review Comment:
   We can copy the doc from the TopicConfig:
   
   ```
   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.
   ```



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-test.jar` file under `storage/build/libs`.

Review Comment:
   Should we ask the user to checkout the 3.6 branch and the build the jar (or) 
from trunk itself?



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1

Review Comment:
   ```suggestion
   # This needs to be changed if number of brokers in the cluster is more than 1
   ```



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>

Review Comment:
   ```suggestion
   <pre>./gradlew clean :storage:testJar</pre>
   ```



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage
+# local.retention.ms=1000 -> the time before local segments are moving to 
remote storage (and then deleted)
+# retention.ms=300000 -> when segments exceed this time, the segments in 
remote storage will be deleted
+# segment.bytes=1000000 -> for test only, to speed up the log segment rolling 
interval
+# file.delete.delay.ms=10000 -> for test only, to speed up the 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=300000 --config 
segment.bytes=1000000 --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 1000 
--record-size 1024 --throughput -1 --producer-props 
bootstrap.servers=localhost:9092

Review Comment:
   If we are changing the `segment.bytes` to MiB format, then the record-size 
might be tuned slightly to ensure that the segment gets rotated.



##########
docs/ops.html:
##########
@@ -3984,27 +3984,73 @@ <h5 class="anchor-heading"><a 
id="tiered_storage_config_topic" class="anchor-lin
   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 `LocalTieredStorage` 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>./gradlew :storage:testJar</pre>
+<p>After build successfully, there should be a 
`kafka-storage-x.x.x-SNAPSHOT-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.
 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=/Users/luke/dev/kafka/storage/build/libs/kafka-storage-x.x.x-SNAPSHOT-test.jar
+
+# Configure the directory used for `LocalTieredStorage`
+rsm.config.dir=/tmp/kafka-remote-storage
+
+# This needs to be changed if broker numbers are more than 1
+rlmm.config.remote.log.metadata.topic.replication.factor=1
 </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>After broker is started, creating a topic with tiered storage enabled with 
configs:
+
+<pre>
+# remote.storage.enable=true -> enabling tiered storage
+# local.retention.ms=1000 -> the time before local segments are moving to 
remote storage (and then deleted)
+# retention.ms=300000 -> when segments exceed this time, the segments in 
remote storage will be deleted
+# segment.bytes=1000000 -> for test only, to speed up the log segment rolling 
interval

Review Comment:
   Can we change the `segment.bytes` to MiB format? 
   
   ```suggestion
   # segment.bytes=1048576 -> for test only, to speed up the log segment 
rolling interval
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to