Hi Henry,

This is a very interesting proposal!
I love the idea to minimize the code change to be able to quickly get
delivered.
Thanks for proposing this!

Some questions:
1. In this KIP, we add one more tier of storage. That is: local disk ->
fast object store -> slow object store.
Why can't we allow users to replace the local disk with the fast object
store directly? Any consideration on this?
If we don't have the local disk, the follower fetch will be much simplified
without downloading from the fast object store, is my understanding correct?

2. Will the WALmetadata be deleted after the data in fast object storage is
deleted?
I'm a little worried about the metadata size in the WALmetadata. I guess
the __remote_log_metadata topic is stored in local disk only, right?

3. In this KIP, we assume the fast object store is different from the slow
object store.
Is it possible we allow users to use the same one?
Let's say, we set both fast/slow object store = S3 (some use cases doesn't
care about too much on the latency), if we offload the active log segment
onto fast object store (S3), can we not offload the segment to slow object
store again after the log segment is rolled?
I'm thinking if it's possible we learn(borrow) some ideas from KIP-1150?
This way, we can achieve the similar goal since we accumulate (combine)
data in multiple partitions and upload to S3 to save the cost.

Thank you.
Luke

On Thu, May 1, 2025 at 1:45 PM Henry Haiying Cai
<haiying_...@yahoo.com.invalid> wrote:

> Link to the KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1176%3A+Tiered+Storage+for+Active+Log+Segment
> Motivation
> In KIP-405, the community has proposed and implemented the tiered storage
> for old Kafka log segment files, when the log segments is older than
> local.retention.ms, it becomes eligible to be uploaded to cloud's object
> storage and removed from the local storage thus reducing local storage
> cost.  KIP-405 only uploads older log segments but not the most recent
> active log segments (write-ahead logs). Thus in a typical 3-way replicated
> Kafka cluster, the 2 follower brokers would still need to replicate the
> active log segments from the leader broker. It is common practice to set up
> the 3 brokers in three different AZs to improve the high availability of
> the cluster. This would cause the replications between leader/follower
> brokers to be across AZs which is a significant cost (various studies show
> the across AZ transfer cost typically comprises 50%-60% of the total
> cluster cost). Since all the active log segments are physically present on
> three Kafka Brokers, they still comprise significant resource usage on the
> brokers. The state of the broker is still quite big during node
> replacement, leading to longer node replacement time. KIP-1150 recently
> proposes diskless Kafka topic, but leads to increased latency and a
> significant redesign. In comparison, this proposed KIP maintains identical
> performance for acks=1 producer path, minimizes design changes to Kafka,
> and still slashes cost by an estimated 43%.
>

Reply via email to