>
> I am ok with adding a note in the KIP but the note should say that it has
> an elevated risk for this scenario due to increased probability of having
> an aggressive local cleanup with Tiered Storage.
>
I would like to clarify that there is no elevated risk because of this KIP.
This risk
alrea
The difference between the two scenarios you mentioned is that with Tiered
Storage, the chances of hitting this scenario increases since a user is
likely to have an aggressive setting for local disk data cleanup, which
would not be the case in empty followers catching up in a non-tiered
storage wor
Hi Divij,
The rare scenario we are discussing is similar to an empty follower trying
to catch up with the leader for a topic that is not enabled with tiered
storage. Consider the following steps:
1. Follower requests offset 0 from the leader.
2. Offset 0 is no more valid on the leader as its log
Thank you for your response Abhijeet. You have understood the scenario
correctly. For the purpose of discussion, please consider the latter case
where offset 11 is not available on the leader anymore (it got cleaned
locally since the last tiered offset is 15). In such a case, you
mentioned, the fol
Hi Divij,
Seems like there is some confusion about the new protocol for fetching from
tiered offset.
The scenario you are highlighting is where,
Leader's Log Start Offset = 0
Last Tiered Offset = 10
Following is the sequence of events that will happen:
1. Follower requests offset 0 from the lead
Following up on my previous comment:
An alternative approach could be to have an empty follower start
replication from last-tiered-offset (already available as part of
listOffsets) inclusive. On the leader, we change the logic (based on a
configurable threshold) on when we return OffsetMovedToTier
Hi folks.
I am late to the party but I have a question on the proposal.
How are we preventing a situation such as the following:
1. Empty follower asks leader for 0
2. Leader compares 0 with last-tiered-offset, and responds with 11 (where10
is last-tiered-offset) and a OffsetMovedToTieredExcepti
Thank you all for your comments. As all the comments in the thread are
addressed, I am starting a Vote thread for the KIP. Please have a look.
Regards.
Abhijeet.
On Thu, Apr 25, 2024 at 6:08 PM Luke Chen wrote:
> Hi, Abhijeet,
>
> Thanks for the update.
>
> I have no more comments.
>
> Luke
>
Hi, Abhijeet,
Thanks for the update.
I have no more comments.
Luke
On Thu, Apr 25, 2024 at 4:21 AM Jun Rao wrote:
> Hi, Abhijeet,
>
> Thanks for the updated KIP. It looks good to me.
>
> Jun
>
> On Mon, Apr 22, 2024 at 12:08 PM Abhijeet Kumar <
> abhijeet.cse@gmail.com>
> wrote:
>
> > Hi
Hi, Abhijeet,
Thanks for the updated KIP. It looks good to me.
Jun
On Mon, Apr 22, 2024 at 12:08 PM Abhijeet Kumar
wrote:
> Hi Jun,
>
> Please find my comments inline.
>
>
> On Thu, Apr 18, 2024 at 3:26 AM Jun Rao wrote:
>
> > Hi, Abhijeet,
> >
> > Thanks for the reply.
> >
> > 1. I am wonder
Hi Jun,
Please find my comments inline.
On Thu, Apr 18, 2024 at 3:26 AM Jun Rao wrote:
> Hi, Abhijeet,
>
> Thanks for the reply.
>
> 1. I am wondering if we could achieve the same result by just lowering
> local.retention.ms and local.retention.bytes. This also allows the newly
> started follo
Hi Luke,
Thanks for your comments. Please find my responses inline.
On Tue, Apr 9, 2024 at 2:08 PM Luke Chen wrote:
> Hi Abhijeet,
>
> Thanks for the KIP to improve the tiered storage feature!
>
> Questions:
> 1. We could also get the "pending-upload-offset" and epoch via remote log
> metadata,
Hi, Abhijeet,
Thanks for the reply.
1. I am wondering if we could achieve the same result by just lowering
local.retention.ms and local.retention.bytes. This also allows the newly
started follower to build up the local data before serving the consumer
traffic.
2. Have you updated the KIP?
Thank
+1 to Jun for adding the consumer fetching from a follower scenario
also to the existing section that talked about the drawback when a
node built with last-tiered-offset has become a leader. As Abhijeet
mentioned, we plan to have a follow-up KIP that will address these by
having a deprioritzation o
Hi Abhijeet,
Thanks for the KIP to improve the tiered storage feature!
Questions:
1. We could also get the "pending-upload-offset" and epoch via remote log
metadata, instead of adding a new API to fetch from the leader. Could you
explain why you choose the later approach, instead of the former?
2
Hi Christo,
Please find my comments inline.
On Fri, Apr 5, 2024 at 12:36 PM Christo Lolov
wrote:
> Hello Abhijeet and Jun,
>
> I have been mulling this KIP over a bit more in recent days!
>
> re: Jun
>
> I wasn't aware we apply 2.1 and 2.2 for reserving new timestamps - in
> retrospect it shoul
Hi Jun,
Thank you for taking the time to review the KIP. Please find my comments
inline.
On Fri, Apr 5, 2024 at 12:09 AM Jun Rao wrote:
> Hi, Abhijeet,
>
> Thanks for the KIP. Left a few comments.
>
> 1. "A drawback of using the last-tiered-offset is that this new follower
> would possess only
Hello Abhijeet and Jun,
I have been mulling this KIP over a bit more in recent days!
re: Jun
I wasn't aware we apply 2.1 and 2.2 for reserving new timestamps - in
retrospect it should have been fairly obvious. I would need to go an update
KIP-1005 myself then, thank you for giving the useful ref
Hi, Abhijeet,
Thanks for the KIP. Left a few comments.
1. "A drawback of using the last-tiered-offset is that this new follower
would possess only a limited number of locally stored segments. Should it
ascend to the role of leader, there is a risk of needing to fetch these
segments from the remot
Hi Christo,
Thanks for reviewing the KIP.
The follower needs the earliest-pending-upload-offset (and the
corresponding leader epoch) from the leader.
This is the first offset the follower will have locally.
Regards,
Abhijeet.
On Fri, Mar 29, 2024 at 1:14 PM Christo Lolov
wrote:
> Heya!
>
>
Heya!
First of all, thank you very much for the proposal, you have explained the
problem you want solved very well - I think a faster bootstrap of an empty
replica is definitely an improvement!
For my understanding, which concrete offset do you want the leader to give
back to a follower - earlies
Hi All,
I have created KIP-1023 to introduce follower fetch from tiered offset.
This feature will be helpful in significantly reducing Kafka
rebalance/rebuild times when the cluster is enabled with tiered storage.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1023%3A+Follower+fetch+from+t
22 matches
Mail list logo