Thanks Jorge. I have updated the KIP based on your comments and see answers
below inline for JQ2/JQ5/end-paragraph.
On Monday, June 2, 2025 at 02:52:38 AM PDT, Jorge Esteban Quilcate Otoya
wrote:
Thanks for clarifying! Things look more clear from my end.
A couple more comments inline:
Thanks for clarifying! Things look more clear from my end.
A couple more comments inline:
On Thu, 29 May 2025 at 10:02, Henry Haiying Cai
wrote:
> Jorge,
> Thanks for your detailed questions, please see my answer inline below.
> On Tuesday, May 27, 2025 at 12:48:01 AM PDT, Jorge Esteban Qui
Jorge,
Thanks for your detailed questions, please see my answer inline below.
On Tuesday, May 27, 2025 at 12:48:01 AM PDT, Jorge Esteban Quilcate Otoya
wrote:
Hi Henry,
Thanks for the proposal and the effort put on this!
I have some comments on the KIP and the ongoing discussion:
JQ1
Hi Henry,
Thanks for the proposal and the effort put on this!
I have some comments on the KIP and the ongoing discussion:
JQ1. In the Motivation section is stated:
> when the log segments is older than local.retention.ms, it becomes
eligible to be uploaded to cloud's object storage and removed
Yes Stan,
That issue (modify FetchRequest/Response API to carry the extra metadata for
the active log segment in object storage) was discussed with Luke, in case that
apache email thread is difficult to read, here is what I said earlier:
HC>. I like your idea of using FetchResponse to carry the
Hi Stanislav,
I already gave a similar suggestion to Henry earlier, and you can see his
response here:
https://lists.apache.org/thread/v8t7co0517hw2tlm0ypn8tnjfmhnhv83 .
Good to see you have the same thought. :)
Thanks.
Luke
On Wed, May 14, 2025 at 6:31 PM Stanislav Kozlovski <
stanislavkozlov.
Have we considered using the traditional replication path to store the actual
metadata for the topic/partition?
I know the KIP says "The main purpose of
FollowerFetchRequest/FollowerFetchResponse is now just to update the offsets
and high watermark between leader and follower.", but what if we
Luke,
Thanks for the comments. Yes I agree we should have 2 metadata topic (one for
closed log segment, one for active log segment) since the lifecycle and
retention policy is going to be very different. Since 2 metadata topic would
require code factoring on ConsumerManager/ProducerManager/Me
Hi Henry,
> Thanks for the detailed response again. I added a section:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1176%3A+Tiered+Storage+for+Active+Log+Segment#KIP1176:TieredStorageforActiveLogSegment-ComparisontoKIP-405:TieredStorage
to
compare this KIP and KIP-405 for some of the poi
Luke,
Thanks for the detailed response again. I added a section:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1176%3A+Tiered+Storage+for+Active+Log+Segment#KIP1176:TieredStorageforActiveLogSegment-ComparisontoKIP-405:TieredStorage
to compare this KIP and KIP-405 for some of the points
Hi Henry,
Thanks for the response.
1. HC> Yes we only upload data logs to fast cloud storage. The follower
broker will build up its own indexes, producer snapshot as it appends data
records to its local logs (in the same way today when the follower builds
up those indexing data structure when it
Luke,
Thanks again for detailed comments and suggestions, see my answer below inline
with HC> indentation.
On Thursday, May 8, 2025 at 12:59:43 AM PDT, Luke Chen
wrote:
Hi Xinyu and Henry,
I think the WAL metadata in KIP1176 is not for log recover, the log
recovery still loads log seg
Luke,
Thanks for your continued suggestions on this KIP, see an my answer below
inline with HC> indentation.
On Thursday, May 8, 2025 at 02:10:07 AM PDT, Luke Chen
wrote:
Hi Henry,
Some more questions:
2. It seems in the KIP, I _assume_ the fast cloud storage will always be
available
Xinyu,
Thanks for your time and comments again. For the metadata management using a
Kafka topic, we are basically following what KIP-405 was doing by reusing the
same metadata topic and the same metadata cache based on that topic.
It's also doable by scanning the WAL in S3 to get the metadata,
Hi Henry,
Some more questions:
2. It seems in the KIP, I _assume_ the fast cloud storage will always be
available without issue.
What will happen when the fast cloud storage is down?
Do we fail the producer write immediately? Or do we have any fall back
mechanism?
Maybe we can fall back to the tra
Hi Xinyu and Henry,
I think the WAL metadata in KIP1176 is not for log recover, the log
recovery still loads log segments locally.
The WAL metadata is for leader <-> follower information sharing only. Is my
understanding correct?
About the WAL metadata, as I mentioned earlier, I still worry about
Hi Henry,
Thank you for your detailed reply. The answer makes sense to me, and you're
right, KIP-1176 has a clear and specific scope and is expected to have a
quick path to implement it.
I also want to discuss the metadata management of WAL log segments. Is an
internal topic necessary for managin
Xinyu,
Thanks for your time reading the KIP and detailed comments. We are honored to
have technical leaders from AutoMQ to look at our work.
Please see my answers below inline.
On Tuesday, May 6, 2025 at 08:37:22 PM PDT, Xinyu Zhou
wrote:
Hi Henry and Tom,
I've read the entire KIP-1
Hi Henry and Tom,
I've read the entire KIP-1176, and I think it's a smart move to advance
tiered storage.
If I understand correctly, KIP-1176 aims to eliminate cross-AZ traffic in
tier 1 storage by replicating data to followers through the S3EOZ bucket.
After that, followers only need to replicat
Christo,
In terms of supporting transactional messages, I looked at the current
FetchRequest/Response code, looks like for follower fetch it's always fetching
to the LOG_END offset (while for consumer fetch there is a choice of fetch up
to HIGH_WATERMARK vs fetch up to TXN_COMMITTED) , since o
Christo,
Thanks for your detailed comments and see my answer below inline.
On Tuesday, May 6, 2025 at 02:40:29 AM PDT, Christo Lolov
wrote:
Hello!
It is great to see another proposal on the same topic, but optimising for
different scenarios, so thanks a lot for the effort put in this!
Hi Christo,
Thanks for your feedback! Responding as a co-author to some of these items:
For acks=-1, it's true with AZ failure S3E1Z data could be lost. In this
case, we can use option 1 "The leader broker will wait until all in-sync
follower replicas have reached the given offset before the ackn
Hello!
It is great to see another proposal on the same topic, but optimising for
different scenarios, so thanks a lot for the effort put in this!
I have a few questions and statements in no particular order.
If you use acks=-1 (acks=all) then an acknowledgement can only be sent to
the producer i
Luke,
Thanks for your comments, see my answers below inline.
On Thursday, May 1, 2025 at 03:20:54 AM PDT, 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 qu
See answers below
On Thursday, May 1, 2025 at 06:14:34 AM PDT, Stanislav Kozlovski
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
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 n
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 a
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 be
28 matches
Mail list logo