Heya!

re: Luke

1. I am not certain I follow the question. From DISABLED you can only go to
ENABLED regardless of whether your cluster is backed by Zookeeper or KRaft.
Am I misunderstanding your point?

2. Apologies, this was a leftover from previous versions. I have updated
the Zookeeper section. The steps ought to be: controller receives change,
commits necessary data to Zookeeper, enqueues disablement and starts
sending StopReplicas request to brokers; brokers receive StopReplicas and
propagate them all the way to RemoteLogManager#stopPartitions which takes
care of the rest.

3. Correct, it should say DISABLED - this should now be corrected.

4. I was thinking that if there is a mismatch we will just fail accepting
the request for disablement. This should be the same in both Zookeeper and
KRaft. Or am I misunderstanding your question?

5. Yeah. I am now doing a second pass on all diagrams and will update them
by the end of the day!

6. I think my current train of thought is that there will be unlimited
retries until all brokers respond in a similar way to how deletion of a
topic works today in ZK. In the meantime the state will continue to be
DISABLING. Do you have a better suggestion?

re: Kamal

Yep, I will update all diagrams

I am not certain I follow the reasoning for making retain and delete the
same. Deletion when the policy is retain happens asynchronously due to
expiration. I think that deletion when the policy is delete ought to (at
least for the initial implementation) happen synchronously. Should people
run into timeout problems we can always then have a follow-up KIP where we
make it asynchronous.

Best,
Christo

On Tue, 7 May 2024 at 10:04, Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> Hi Christo,
>
> Thanks for the update!
>
> For both the policies "retain" and "delete", can we maintain the same
> approach to delete the segments async?
>
> > If the disablement policy is set to delete, the Log start offset (LSO) is
> updated to match the Local Log Start Offset and the remote log is deleted
> by calling the RemoteStorageManager#deleteLogSegmentData().
>
> In the KIP, it's mentioned that when the disable policy is set to "delete",
> the remote-log-segments will be
> deleted in-sync. The stopPartition call might get timed out when the number
> of remote log segments to
> delete is huge. We can further extend the same approach for the topic
> deletion requests.
>
> Also, Could you please update the state diagram about the transitions? It
> is not clear when to transit from
> DISABLING to DISABLED state?
>
> --
> Kamal
>
> On Mon, May 6, 2024 at 6:55 PM Kamal Chandraprakash <
> kamal.chandraprak...@gmail.com> wrote:
>
> > Ignore the above message. Got the answers after reading the state
> > transition section.
> >
> > > If the disablement policy is delete, tasks scheduled for the
> > topic-partitions in the RemoteDataExpirationThreadPool will also be
> > canceled.
> >
> > We are deleting the segments synchronously. Should we delete them
> > asynchronously? The same approach can be extended to topic deletion
> > requests.
> >
> > > 6. In ZK mode, what will the controller do if the "stopReplicas"
> > responses not received from all brokers? Reverting the changes?
> >
> > Since we are deleting the segments synchronously. This case can be bound
> > to happen when the number of remote log segments to
> > delete is huge.
> >
> >
> > On Mon, May 6, 2024, 18:12 Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> >> Hi Christo,
> >>
> >> Thanks for the update!
> >>
> >> 1. In the ZK mode, how will the transition from DISABLING to DISABLED
> >> state happen?
> >> For the "retain" policy, until we delete all the remote-log segments,
> the
> >> state will be
> >> DISABLING and the deletion can happen only when they breach either the
> >> retention
> >> time (or) size.
> >>
> >> How does the controller monitor that all the remote log segments are
> >> deleted for all
> >> the partitions of the topic before transitioning the state to DISABLED?
> >>
> >> 2. In Kraft, we have only ENABLED -> DISABLED state. How are we
> >> supporting the case
> >> "retain" -> "enable"?
> >>
> >> If the remote storage is degraded, we want to avoid uploading the
> >> segments temporarily
> >> and resume back once the remote storage is healthy. Is the case
> supported?
> >>
> >>
> >>
> >> On Fri, May 3, 2024 at 12:12 PM Luke Chen <show...@gmail.com> wrote:
> >>
> >>> Also, I think using `stopReplicas` request is a good idea because it
> >>> won't cause any problems while migrating to KRaft mode.
> >>> The stopReplicas request is one of the request that KRaft controller
> >>> will send to ZK brokers during migration.
> >>>
> >>> Thanks.
> >>> Luke
> >>>
> >>> On Fri, May 3, 2024 at 11:48 AM Luke Chen <show...@gmail.com> wrote:
> >>>
> >>>> Hi Christo,
> >>>>
> >>>> Thanks for the update.
> >>>>
> >>>> Questions:
> >>>> 1. For this
> >>>> "The possible state transition from DISABLED state is to the ENABLED."
> >>>> I think it only applies for KRaft mode. In ZK mode, the possible state
> >>>> is "DISABLING", right?
> >>>>
> >>>> 2. For this:
> >>>> "If the cluster is using Zookeeper as the control plane, enabling
> >>>> remote storage for a topic triggers the controller to send this
> information
> >>>> to Zookeeper. Each broker listens for changes in Zookeeper, and when a
> >>>> change is detected, the broker triggers
> >>>> RemoteLogManager#onLeadershipChange()."
> >>>>
> >>>> I think the way ZK brokers knows the leadership change is by getting
> >>>> the LeaderAndISRRequeset from the controller, not listening for
> changes in
> >>>> ZK.
> >>>>
> >>>> 3. In the KRaft handler steps, you said:
> >>>> "The controller also updates the Topic metadata to increment the
> >>>> tiered_epoch and update the tiered_stateto DISABLING state."
> >>>>
> >>>> Should it be "DISABLED" state since it's KRaft mode?
> >>>>
> >>>> 4. I was thinking how we handle the tiered_epoch not match error.
> >>>> For ZK, I think the controller won't write any data into ZK Znode,
> >>>> For KRaft, either configRecord or updateTopicMetadata records won't be
> >>>> written.
> >>>> Is that right? Because the current workflow makes me think there will
> >>>> be partial data updated in ZK/KRaft when tiered_epoch error.
> >>>>
> >>>> 5. Since we changed to use stopReplicas (V5) request now, the diagram
> >>>> for ZK workflow might also need to update.
> >>>>
> >>>> 6. In ZK mode, what will the controller do if the "stopReplicas"
> >>>> responses not received from all brokers? Reverting the changes?
> >>>> This won't happen in KRaft mode because it's broker's responsibility
> to
> >>>> fetch metadata update from controller.
> >>>>
> >>>>
> >>>> Thank you.
> >>>> Luke
> >>>>
> >>>>
> >>>> On Fri, Apr 19, 2024 at 10:23 PM Christo Lolov <
> christolo...@gmail.com>
> >>>> wrote:
> >>>>
> >>>>> Heya all!
> >>>>>
> >>>>> I have updated KIP-950. A list of what I have updated is:
> >>>>>
> >>>>> * Explicitly state that Zookeeper-backed clusters will have ENABLED
> ->
> >>>>> DISABLING -> DISABLED while KRaft-backed clusters will only have
> ENABLED ->
> >>>>> DISABLED
> >>>>> * Added two configurations for the new thread pools and explained
> >>>>> where values will be picked-up mid Kafka version upgrade
> >>>>> * Explained how leftover remote partitions will be scheduled for
> >>>>> deletion
> >>>>> * Updated the API to use StopReplica V5 rather than a whole new
> >>>>> controller-to-broker API
> >>>>> * Explained that the disablement procedure will be triggered by the
> >>>>> controller listening for an (Incremental)AlterConfig change
> >>>>> * Explained that we will first move log start offset and then issue a
> >>>>> deletion
> >>>>> * Went into more details that changing remote.log.disable.policy
> after
> >>>>> disablement won't do anything and that if a customer would like
> additional
> >>>>> data deleted they would have to use already existing methods
> >>>>>
> >>>>> Let me know if there are any new comments or I have missed something!
> >>>>>
> >>>>> Best,
> >>>>> Christo
> >>>>>
> >>>>> On Mon, 15 Apr 2024 at 12:40, Christo Lolov <christolo...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Heya Doguscan,
> >>>>>>
> >>>>>> I believe that the state of the world after this KIP will be the
> >>>>>> following:
> >>>>>>
> >>>>>> For Zookeeper-backed clusters there will be 3 states: ENABLED,
> >>>>>> DISABLING and DISABLED. We want this because Zookeeper-backed
> clusters will
> >>>>>> await a confirmation from the brokers that they have indeed stopped
> >>>>>> tiered-related operations on the topic.
> >>>>>>
> >>>>>> For KRaft-backed clusters there will be only 2 states: ENABLED and
> >>>>>> DISABLED. KRaft takes a fire-and-forget approach for topic
> deletion. I
> >>>>>> believe the same approach ought to be taken for tiered topics. The
> >>>>>> mechanism which will ensure that leftover state in remote due to
> failures
> >>>>>> is cleaned up to me is the retention mechanism. In today's code, a
> leader
> >>>>>> deletes all segments it finds in remote with offsets below the log
> start
> >>>>>> offset. I believe this will be good enough for cleaning up leftover
> state
> >>>>>> in remote due to failures.
> >>>>>>
> >>>>>> I know that quite a few changes have been discussed so I will aim to
> >>>>>> put them on paper in the upcoming days and let everyone know!
> >>>>>>
> >>>>>> Best,
> >>>>>> Christo
> >>>>>>
> >>>>>> On Tue, 9 Apr 2024 at 14:49, Doğuşcan Namal <
> namal.dogus...@gmail.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> +1 let's not introduce a new api and mark it immediately as
> >>>>>>> deprecated :)
> >>>>>>>
> >>>>>>> On your second comment Luke, one thing we need to clarify is when
> do
> >>>>>>> we consider remote storage to be DISABLED for a topic?
> >>>>>>> Particularly, what is the state when the remote storage is being
> >>>>>>> deleted in case of disablement.policy=delete? Is it DISABLING or
> DISABLED?
> >>>>>>>
> >>>>>>> If we move directly to the DISABLED state,
> >>>>>>>
> >>>>>>> a) in case of failures, the leaders should continue remote storage
> >>>>>>> deletion even if the topic is moved to the DISABLED state,
> otherwise we
> >>>>>>> risk having stray data on remote storage.
> >>>>>>> b) on each restart, we should initiate the remote storage deletion
> >>>>>>> because although we replayed a record with a DISABLED state, we
> can not be
> >>>>>>> sure if the remote data is deleted or not.
> >>>>>>>
> >>>>>>> We could either consider keeping the remote topic in DISABLING
> state
> >>>>>>> until all of the remote storage data is deleted, or we need an
> additional
> >>>>>>> mechanism to handle the remote stray data.
> >>>>>>>
> >>>>>>> The existing topic deletion, for instance, handles stray logs on
> >>>>>>> disk by detecting them on KafkaBroker startup and deleting before
> the
> >>>>>>> ReplicaManager is started.
> >>>>>>> Maybe we need a similar mechanism here as well if we don't want a
> >>>>>>> DISABLING state. Otherwise, we need a callback from Brokers to
> validate
> >>>>>>> that remote storage data is deleted and now we could move to the
> DISABLED
> >>>>>>> state.
> >>>>>>>
> >>>>>>> Thanks.
> >>>>>>>
> >>>>>>> On Tue, 9 Apr 2024 at 12:45, Luke Chen <show...@gmail.com> wrote:
> >>>>>>>
> >>>>>>>> Hi Christo,
> >>>>>>>>
> >>>>>>>> > I would then opt for moving information from DisableRemoteTopic
> >>>>>>>> within the StopReplicas API which will then disappear in KRaft
> >>>>>>>> world as it
> >>>>>>>> is already scheduled for deprecation. What do you think?
> >>>>>>>>
> >>>>>>>> Sounds good to me.
> >>>>>>>>
> >>>>>>>> Thanks.
> >>>>>>>> Luke
> >>>>>>>>
> >>>>>>>> On Tue, Apr 9, 2024 at 6:46 PM Christo Lolov <
> >>>>>>>> christolo...@gmail.com> wrote:
> >>>>>>>>
> >>>>>>>> > Heya Luke!
> >>>>>>>> >
> >>>>>>>> > I thought a bit more about it and I reached the same conclusion
> >>>>>>>> as you for
> >>>>>>>> > 2 as a follow-up from 1. In other words, in KRaft world I don't
> >>>>>>>> think the
> >>>>>>>> > controller needs to wait for acknowledgements for the brokers.
> >>>>>>>> All we care
> >>>>>>>> > about is that the leader (who is responsible for
> >>>>>>>> archiving/deleting data in
> >>>>>>>> > tiered storage) knows about the change and applies it properly.
> >>>>>>>> If there is
> >>>>>>>> > a leadership change halfway through the operation then the new
> >>>>>>>> leader still
> >>>>>>>> > needs to apply the message from the state topic and we know
> that a
> >>>>>>>> > disable-message will be applied before a reenablement-message. I
> >>>>>>>> will
> >>>>>>>> > change the KIP later today/tomorrow morning to reflect this
> >>>>>>>> reasoning.
> >>>>>>>> >
> >>>>>>>> > However, with this I believe that introducing a new API just for
> >>>>>>>> > Zookeeper-based clusters (i.e. DisableRemoteTopic) becomes a bit
> >>>>>>>> of an
> >>>>>>>> > overkill. I would then opt for moving information from
> >>>>>>>> DisableRemoteTopic
> >>>>>>>> > within the StopReplicas API which will then disappear in KRaft
> >>>>>>>> world as it
> >>>>>>>> > is already scheduled for deprecation. What do you think?
> >>>>>>>> >
> >>>>>>>> > Best,
> >>>>>>>> > Christo
> >>>>>>>> >
> >>>>>>>> > On Wed, 3 Apr 2024 at 07:59, Luke Chen <show...@gmail.com>
> wrote:
> >>>>>>>> >
> >>>>>>>> > > Hi Christo,
> >>>>>>>> > >
> >>>>>>>> > > 1. I agree with Doguscan that in KRaft mode, the controller
> >>>>>>>> won't send
> >>>>>>>> > RPCs
> >>>>>>>> > > to the brokers (except in the migration path).
> >>>>>>>> > > So, I think we could adopt the similar way we did to
> >>>>>>>> > `AlterReplicaLogDirs`
> >>>>>>>> > > (
> >>>>>>>> > > KIP-858
> >>>>>>>> > > <
> >>>>>>>> > >
> >>>>>>>> >
> >>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-858%3A+Handle+JBOD+broker+disk+failure+in+KRaft#KIP858:HandleJBODbrokerdiskfailureinKRaft-Intra-brokerreplicamovement
> >>>>>>>> > > >)
> >>>>>>>> > > that let the broker notify controller any update, instead of
> >>>>>>>> controller
> >>>>>>>> > to
> >>>>>>>> > > broker. And once the controller receives all the complete
> >>>>>>>> requests from
> >>>>>>>> > > brokers, it'll enter "Disabled" state. WDYT?
> >>>>>>>> > >
> >>>>>>>> > > 2. Why should we wait until all brokers to respond before
> >>>>>>>> moving to
> >>>>>>>> > > "Disabled" state in "KRaft mode"?
> >>>>>>>> > > Currently, only the leader node does the remote log
> >>>>>>>> upload/fetch tasks,
> >>>>>>>> > so
> >>>>>>>> > > does that mean the controller only need to make sure the
> leader
> >>>>>>>> completes
> >>>>>>>> > > the stopPartition?
> >>>>>>>> > > If during the leader node stopPartition process triggered
> >>>>>>>> leadership
> >>>>>>>> > > change, then the new leader should receive and apply the
> >>>>>>>> configRecord
> >>>>>>>> > > update before the leadership change record based on the KRaft
> >>>>>>>> design,
> >>>>>>>> > which
> >>>>>>>> > > means there will be no gap that the follower node becomes the
> >>>>>>>> leader and
> >>>>>>>> > > starting doing unexpected upload/fetch tasks, right?
> >>>>>>>> > > I agree we should make sure in ZK mode, all brokers are
> >>>>>>>> completed the
> >>>>>>>> > > stopPartitions before moving to "Disabled" state because ZK
> >>>>>>>> node watcher
> >>>>>>>> > is
> >>>>>>>> > > working in a separate thread. But not sure about KRaft mode.
> >>>>>>>> > >
> >>>>>>>> > > Thanks.
> >>>>>>>> > > Luke
> >>>>>>>> > >
> >>>>>>>> > >
> >>>>>>>> > > On Fri, Mar 29, 2024 at 4:14 PM Christo Lolov <
> >>>>>>>> christolo...@gmail.com>
> >>>>>>>> > > wrote:
> >>>>>>>> > >
> >>>>>>>> > > > Heya everyone!
> >>>>>>>> > > >
> >>>>>>>> > > > re: Doguscan
> >>>>>>>> > > >
> >>>>>>>> > > > I believe the answer to 101 needs a bit more discussion. As
> >>>>>>>> far as I
> >>>>>>>> > > know,
> >>>>>>>> > > > tiered storage today has methods to update a metadata of a
> >>>>>>>> segment to
> >>>>>>>> > say
> >>>>>>>> > > > "hey, I would like this deleted", but actual deletion is
> left
> >>>>>>>> to plugin
> >>>>>>>> > > > implementations (or any background cleaners). In other
> words,
> >>>>>>>> there is
> >>>>>>>> > no
> >>>>>>>> > > > "immediate" deletion. In this KIP, we would like to continue
> >>>>>>>> doing the
> >>>>>>>> > > same
> >>>>>>>> > > > if the retention policy is set to delete. So I believe the
> >>>>>>>> answer is
> >>>>>>>> > > > actually that a) we will update the metadata of the segments
> >>>>>>>> to mark
> >>>>>>>> > them
> >>>>>>>> > > > as deleted and b) we will advance the log start offset. Any
> >>>>>>>> deletion of
> >>>>>>>> > > > actual files will still be delegated to plugin
> >>>>>>>> implementations. I
> >>>>>>>> > believe
> >>>>>>>> > > > this is further supported by
> >>>>>>>> "*remote.log.disable.policy=delete:* Logs
> >>>>>>>> > > that
> >>>>>>>> > > > are archived in the remote storage will not be part of the
> >>>>>>>> contiguous
> >>>>>>>> > > > "active" log and will be deleted asynchronously as part of
> the
> >>>>>>>> > > disablement
> >>>>>>>> > > > process"
> >>>>>>>> > > >
> >>>>>>>> > > > Following from the above, I believe for 102 it is fine to
> >>>>>>>> allow setting
> >>>>>>>> > > of
> >>>>>>>> > > > remote.log.disable.policy on a disabled topic in much the
> >>>>>>>> same way we
> >>>>>>>> > > allow
> >>>>>>>> > > > other remote-related configurations to be set on a topic
> (i.e.
> >>>>>>>> > > > local.retention.*) - it just won't have an effect. Granted,
> I
> >>>>>>>> do
> >>>>>>>> > believe
> >>>>>>>> > > we
> >>>>>>>> > > > should restrict the policy being changed while a disablement
> >>>>>>>> is
> >>>>>>>> > ongoing.
> >>>>>>>> > > >
> >>>>>>>> > > > re: Satish and Kamal
> >>>>>>>> > > >
> >>>>>>>> > > > 104, 1 and 2 are fair asks, I will work with Doguscan to
> >>>>>>>> update the KIP
> >>>>>>>> > > > with the information!
> >>>>>>>> > > >
> >>>>>>>> > > > Best,
> >>>>>>>> > > > Christo
> >>>>>>>> > > >
> >>>>>>>> > > > On Thu, 28 Mar 2024 at 10:31, Doğuşcan Namal <
> >>>>>>>> namal.dogus...@gmail.com
> >>>>>>>> > >
> >>>>>>>> > > > wrote:
> >>>>>>>> > > >
> >>>>>>>> > > > > Hi Satish, I will try to answer as much as I can and the
> >>>>>>>> others could
> >>>>>>>> > > > chime
> >>>>>>>> > > > > in with further details.
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > > *101. For remote.log.disable.policy=delete: Does it delete
> >>>>>>>> the remote
> >>>>>>>> > > log
> >>>>>>>> > > > > data immediately and the data in remote storage will not
> be
> >>>>>>>> taken
> >>>>>>>> > into
> >>>>>>>> > > > > account by any replica? That means log-start-offset is
> >>>>>>>> moved to the
> >>>>>>>> > > > earlier
> >>>>>>>> > > > > local-log-start-offset.*
> >>>>>>>> > > > > *Exactly. RemoteLogData will be deleted immediately. *
> >>>>>>>> > > > > *So before the deletion starts we move LogStart offset to
> >>>>>>>> > > > > LocalLogStartOffset to ensure that no RemoteLog will be
> >>>>>>>> accessed
> >>>>>>>> > after
> >>>>>>>> > > > that
> >>>>>>>> > > > > point.*
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > > * 102. Can we update the remote.log.disable.policy after
> >>>>>>>> tiered
> >>>>>>>> > storage
> >>>>>>>> > > > is
> >>>>>>>> > > > > disabled on a topic?*
> >>>>>>>> > > > >
> >>>>>>>> > > > > *This is a good point. I think we should not allow
> >>>>>>>> modifying this
> >>>>>>>> > > > > configuration*
> >>>>>>>> > > > > *because changing the policy from Deletion to Retain when
> >>>>>>>> there is an
> >>>>>>>> > > > > ongoing Deletion will result in an undefined behaviour and
> >>>>>>>> where we
> >>>>>>>> > > > retain
> >>>>>>>> > > > > half of the remote log and delete the other half.*
> >>>>>>>> > > > >
> >>>>>>>> > > > > * 103. Do we plan to add any metrics related to this
> >>>>>>>> feature?*
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > > *Any recommendations?*
> >>>>>>>> > > > > *We may emit a gauge showing the enablement state of a
> >>>>>>>> topic but we
> >>>>>>>> > > could
> >>>>>>>> > > > > gather that info from the logs as well.*
> >>>>>>>> > > > > *The total duration for remote topic deletion could be
> >>>>>>>> added as well
> >>>>>>>> > > but
> >>>>>>>> > > > > this is more of a metric for the RemotePartitionRemover
> >>>>>>>> itself.*
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > > *104. Please add configuration details about copier thread
> >>>>>>>> pool,
> >>>>>>>> > > > expiration
> >>>>>>>> > > > > thread pool and the migration of the existing
> >>>>>>>> > > > > RemoteLogManagerScheduledThreadPool.*
> >>>>>>>> > > > >
> >>>>>>>> > > > > *Will add the details.*
> >>>>>>>> > > > >
> >>>>>>>> > > > > 105. How is the behaviour with topic or partition deletion
> >>>>>>>> request
> >>>>>>>> > > > > handled when tiered storage disablement request is still
> >>>>>>>> being
> >>>>>>>> > > > > processed on a topic?
> >>>>>>>> > > > >
> >>>>>>>> > > > > *If the disablement policy is Delete then a successive
> >>>>>>>> topic deletion
> >>>>>>>> > > > > request is going to be a NOOP because RemoteLogs is
> already
> >>>>>>>> being
> >>>>>>>> > > > deleted.*
> >>>>>>>> > > > > *If the disablement policy is Retain, then we only moved
> the
> >>>>>>>> > > > LogStartOffset
> >>>>>>>> > > > > and didn't touch RemoteLogs anyway, so the delete topic
> >>>>>>>> request will
> >>>>>>>> > > > > result*
> >>>>>>>> > > > >
> >>>>>>>> > > > > *in the initiation of RemoteLog deletion.*
> >>>>>>>> > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > > > On Tue, 26 Mar 2024 at 18:21, Kamal Chandraprakash <
> >>>>>>>> > > > > kamal.chandraprak...@gmail.com> wrote:
> >>>>>>>> > > > >
> >>>>>>>> > > > > > Hi,
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > Thanks for the KIP! Overall the KIP looks good and
> >>>>>>>> covered most of
> >>>>>>>> > > the
> >>>>>>>> > > > > > items.
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > 1. Could you explain how the brokers will handle the
> >>>>>>>> > > DisableRemoteTopic
> >>>>>>>> > > > > API
> >>>>>>>> > > > > > request?
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > 2. Who will initiate the controller interaction
> sequence?
> >>>>>>>> Does the
> >>>>>>>> > > > > > controller listens for
> >>>>>>>> > > > > > topic config updates and initiate the disablement?
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > --
> >>>>>>>> > > > > > Kamal
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > On Tue, Mar 26, 2024 at 4:40 PM Satish Duggana <
> >>>>>>>> > > > satish.dugg...@gmail.com
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > wrote:
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > > Thanks Mehari, Divij, Christo etal for the KIP.
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > I had an initial review of the KIP and left the below
> >>>>>>>> comments.
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > 101. For remote.log.disable.policy=delete:
> >>>>>>>> > > > > > > Does it delete the remote log data immediately and the
> >>>>>>>> data in
> >>>>>>>> > > remote
> >>>>>>>> > > > > > > storage will not be taken into account by any replica?
> >>>>>>>> That means
> >>>>>>>> > > > > > > log-start-offset is moved to the earlier
> >>>>>>>> local-log-start-offset.
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > 102. Can we update the remote.log.disable.policy after
> >>>>>>>> tiered
> >>>>>>>> > > storage
> >>>>>>>> > > > > > > is disabled on a topic?
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > 103. Do we plan to add any metrics related to this
> >>>>>>>> feature?
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > 104. Please add configuration details about copier
> >>>>>>>> thread pool,
> >>>>>>>> > > > > > > expiration thread pool and the migration of the
> existing
> >>>>>>>> > > > > > > RemoteLogManagerScheduledThreadPool.
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > 105. How is the behaviour with topic or partition
> >>>>>>>> deletion
> >>>>>>>> > request
> >>>>>>>> > > > > > > handled when tiered storage disablement request is
> >>>>>>>> still being
> >>>>>>>> > > > > > > processed on a topic?
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > ~Satish.
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > > > On Mon, 25 Mar 2024 at 13:34, Doğuşcan Namal <
> >>>>>>>> > > > namal.dogus...@gmail.com
> >>>>>>>> > > > > >
> >>>>>>>> > > > > > > wrote:
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > Hi Christo and Luke,
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > I think the KRaft section of the KIP requires slight
> >>>>>>>> > improvement.
> >>>>>>>> > > > The
> >>>>>>>> > > > > > > metadata propagation in KRaft is handled by the RAFT
> >>>>>>>> layer
> >>>>>>>> > instead
> >>>>>>>> > > of
> >>>>>>>> > > > > > > sending Controller -> Broker RPCs. In fact, KIP-631
> >>>>>>>> deprecated
> >>>>>>>> > > these
> >>>>>>>> > > > > > RPCs.
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > I will come up with some recommendations on how we
> >>>>>>>> could
> >>>>>>>> > improve
> >>>>>>>> > > > that
> >>>>>>>> > > > > > > one but until then, @Luke please feel free to review
> >>>>>>>> the KIP.
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > @Satish, if we want this to make it to Kafka 3.8 I
> >>>>>>>> believe we
> >>>>>>>> > > need
> >>>>>>>> > > > to
> >>>>>>>> > > > > > > aim to get the KIP approved in the following weeks
> >>>>>>>> otherwise it
> >>>>>>>> > > will
> >>>>>>>> > > > > slip
> >>>>>>>> > > > > > > and we can not support it in Zookeeper mode.
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > I also would like to better understand what is the
> >>>>>>>> community's
> >>>>>>>> > > > stand
> >>>>>>>> > > > > > for
> >>>>>>>> > > > > > > adding a new feature for Zookeeper since it is marked
> as
> >>>>>>>> > deprecated
> >>>>>>>> > > > > > already.
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > Thanks.
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > > On Mon, 18 Mar 2024 at 13:42, Christo Lolov <
> >>>>>>>> > > > christolo...@gmail.com>
> >>>>>>>> > > > > > > wrote:
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> Heya,
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> I do have some time to put into this, but to be
> >>>>>>>> honest I am
> >>>>>>>> > > still
> >>>>>>>> > > > > > after
> >>>>>>>> > > > > > > >> reviews of the KIP itself :)
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> After the latest changes it ought to be detailing
> >>>>>>>> both a
> >>>>>>>> > > Zookeeper
> >>>>>>>> > > > > > > approach
> >>>>>>>> > > > > > > >> and a KRaft approach.
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> Do you have any thoughts on how it could be
> improved
> >>>>>>>> or
> >>>>>>>> > should I
> >>>>>>>> > > > > > start a
> >>>>>>>> > > > > > > >> voting thread?
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> Best,
> >>>>>>>> > > > > > > >> Christo
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> On Thu, 14 Mar 2024 at 06:12, Luke Chen <
> >>>>>>>> show...@gmail.com>
> >>>>>>>> > > > wrote:
> >>>>>>>> > > > > > > >>
> >>>>>>>> > > > > > > >> > Hi Christo,
> >>>>>>>> > > > > > > >> >
> >>>>>>>> > > > > > > >> > Any update with this KIP?
> >>>>>>>> > > > > > > >> > If you don't have time to complete it, I can
> >>>>>>>> collaborate
> >>>>>>>> > with
> >>>>>>>> > > > you
> >>>>>>>> > > > > to
> >>>>>>>> > > > > > > work
> >>>>>>>> > > > > > > >> > on it.
> >>>>>>>> > > > > > > >> >
> >>>>>>>> > > > > > > >> > Thanks.
> >>>>>>>> > > > > > > >> > Luke
> >>>>>>>> > > > > > > >> >
> >>>>>>>> > > > > > > >> > On Wed, Jan 17, 2024 at 11:38 PM Satish Duggana <
> >>>>>>>> > > > > > > satish.dugg...@gmail.com>
> >>>>>>>> > > > > > > >> > wrote:
> >>>>>>>> > > > > > > >> >
> >>>>>>>> > > > > > > >> > > Hi Christo,
> >>>>>>>> > > > > > > >> > > Thanks for volunteering to contribute to the
> KIP
> >>>>>>>> > > discussion. I
> >>>>>>>> > > > > > > suggest
> >>>>>>>> > > > > > > >> > > considering this KIP for both ZK and KRaft as
> it
> >>>>>>>> will be
> >>>>>>>> > > > helpful
> >>>>>>>> > > > > > for
> >>>>>>>> > > > > > > >> > > this feature to be available in 3.8.0 running
> >>>>>>>> with ZK
> >>>>>>>> > > > clusters.
> >>>>>>>> > > > > > > >> > >
> >>>>>>>> > > > > > > >> > > Thanks,
> >>>>>>>> > > > > > > >> > > Satish.
> >>>>>>>> > > > > > > >> > >
> >>>>>>>> > > > > > > >> > > On Wed, 17 Jan 2024 at 19:04, Christo Lolov <
> >>>>>>>> > > > > > christolo...@gmail.com
> >>>>>>>> > > > > > > >
> >>>>>>>> > > > > > > >> > > wrote:
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > Hello!
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > I volunteer to get this KIP moving forward
> and
> >>>>>>>> > implemented
> >>>>>>>> > > > in
> >>>>>>>> > > > > > > Apache
> >>>>>>>> > > > > > > >> > > Kafka
> >>>>>>>> > > > > > > >> > > > 3.8.
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > I have caught up with Mehari offline and we
> >>>>>>>> have agreed
> >>>>>>>> > > that
> >>>>>>>> > > > > > given
> >>>>>>>> > > > > > > >> > Apache
> >>>>>>>> > > > > > > >> > > > Kafka 4.0 being around the corner we would
> >>>>>>>> like to
> >>>>>>>> > propose
> >>>>>>>> > > > > this
> >>>>>>>> > > > > > > feature
> >>>>>>>> > > > > > > >> > > > only for KRaft clusters.
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > Any and all reviews and comments are welcome!
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > Best,
> >>>>>>>> > > > > > > >> > > > Christo
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > On Tue, 9 Jan 2024 at 09:44, Doğuşcan Namal <
> >>>>>>>> > > > > > > namal.dogus...@gmail.com>
> >>>>>>>> > > > > > > >> > > > wrote:
> >>>>>>>> > > > > > > >> > > >
> >>>>>>>> > > > > > > >> > > > > Hi everyone, any progress on the status of
> >>>>>>>> this KIP?
> >>>>>>>> > > > Overall
> >>>>>>>> > > > > > > looks
> >>>>>>>> > > > > > > >> > > good to
> >>>>>>>> > > > > > > >> > > > > me but I wonder whether we still need to
> >>>>>>>> support it
> >>>>>>>> > for
> >>>>>>>> > > > > > > Zookeeper
> >>>>>>>> > > > > > > >> > mode
> >>>>>>>> > > > > > > >> > > > > given that it will be deprecated in the
> next
> >>>>>>>> 3 months.
> >>>>>>>> > > > > > > >> > > > >
> >>>>>>>> > > > > > > >> > > > > On 2023/07/21 20:16:46 "Beyene, Mehari"
> >>>>>>>> wrote:
> >>>>>>>> > > > > > > >> > > > > > Hi everyone,
> >>>>>>>> > > > > > > >> > > > > > I would like to start a discussion on
> >>>>>>>> KIP-950:
> >>>>>>>> > Tiered
> >>>>>>>> > > > > > Storage
> >>>>>>>> > > > > > > >> > > Disablement
> >>>>>>>> > > > > > > >> > > > > (
> >>>>>>>> > > > > > > >> > > > >
> >>>>>>>> > > > > > > >> > > > >
> >>>>>>>> > > > > > > >> > >
> >>>>>>>> > > > > > > >> >
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > >
> >>>>>>>> > >
> >>>>>>>> >
> >>>>>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> >>>>>>>> > > > > > > >> > > > > ).
> >>>>>>>> > > > > > > >> > > > > >
> >>>>>>>> > > > > > > >> > > > > > This KIP proposes adding the ability to
> >>>>>>>> disable and
> >>>>>>>> > > > > > re-enable
> >>>>>>>> > > > > > > >> > tiered
> >>>>>>>> > > > > > > >> > > > > storage on a topic.
> >>>>>>>> > > > > > > >> > > > > >
> >>>>>>>> > > > > > > >> > > > > > Thanks,
> >>>>>>>> > > > > > > >> > > > > > Mehari
> >>>>>>>> > > > > > > >> > > > > >
> >>>>>>>> > > > > > > >> > > > >
> >>>>>>>> > > > > > > >> > >
> >>>>>>>> > > > > > > >> >
> >>>>>>>> > > > > > >
> >>>>>>>> > > > > >
> >>>>>>>> > > > >
> >>>>>>>> > > >
> >>>>>>>> > >
> >>>>>>>> >
> >>>>>>>>
> >>>>>>>
>

Reply via email to