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