See answers below
    On Thursday, May 1, 2025 at 06:14:34 AM PDT, Stanislav Kozlovski 
<stanislavkozlov...@apache.org> wrote:  
 
 Thanks for the awesome proposal!

Few questions:

1) can we have more detail around the proposed deployment model in GCP/Azure? 
As of reading, the KIP seems overly focused on AWS - but the feature will need 
to have viable support for the other two major clouds, right? In particular it 
would be nice to see cost comparisons there too

HC> We don't have more details in GCP/Azure since our company (Slack) is only 
deploying on AWS.  However based on literature reading, GCP/Azure have similar 
products to compete with AWS's S3E1Z and EBS, e.g. GCS, Google Cloud Hyperdisk 
from Google, Azure Blob Storage, Azure Managed Disk from Azure.  And Azure's 
Blob Storage can be replicated across AZs as well.
2) for the cost benchmark, in the new model, do you have the leader broker, the 
follower broker and the S3E1Z bucket each in a separate zone?
HC> The leader broker and S31EZ bucket are in the same AZ, the follower AZ is 
in a different AZ.

3) for the cost benchmark, in the new model, we are essentially replacing one 
follower broker for S3E1Z. This ends up cheaper in terms of storage/instance, 
because a) there is no third EC2 instance and b) S3E1Z storage ($0.11/GiB) can 
be cheaper than an SSD ($0.08/GiB) when you factor in free space buffers. How 
are you thining about the availability tradeoff we are making here? (2 standby 
brokers vs 3)
HC> The main cost saving is from across AZ traffic cost.  The current proposal 
is still using 2 brokers (one leader and a follower as hot-standby), we do have 
a vision that we can live with just the leader broker and S3E1Z bucket and add 
code to support a fast bootstrap of new leader broker if the old leader 
crashed, that way we can cut down one more broker.   In terms of availability 
tradeoff between our proposal and current Kafka status, having 2 kafka brokers 
in 2 AZs + S3E1Z bucket (also multi-way replicated) has more replicas (probably 
5) comparing to the current Kafka status (3 replicas) although there is one 
more hop (around 10ms) the data needs to travel.  That extra hop of 10ms can be 
further mitigated by tuning batch upload frequency or later enhancement of the 
KIP to remove the 2nd follower broker.

4) the KIP mentions KIP-1150 as a Rejected Alternative, but do you see a reason 
why both cannot co-exist? From my initial read, it seems like both are 
optimized for different use-cases. KIP-1150 is optimized for acks=all requests 
- strict durability requirements with relaxed latency requirements, whereas 
KIP-1176 seems optimized for acks=1 requests - relaxed durability requiremend 
with strict latency requirements

HC> We do view KIP-1176 and KIP-1150 can co-exist, they are optimize for 
different use cases or some of the features/ideas can be merged.  We put 
KIP-1150 in Rejected Alternatives since that's from the KIP template (you need 
to have a rejected alternative in your proposal), I can modify our proposal to 
emphasize that KIP-1150 and KIP-1176 are not mutually exclusive.

Best,
Stanislav

On 2025/05/01 10:20:26 Luke Chen wrote:
> 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