Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Kamal Chandraprakash
Correction:

(2): Wait for all the remote segments to be deleted async due to breach by
retention time (or) size,
   then set the `remote.storage.enable = false` and
`remote.log.delete.on.disable = true`. This step is optional.

On Thu, Jul 25, 2024 at 11:13 PM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> Hi Chia-Ping,
>
> Thanks for the review!
>
> >  If so, what is the purpose of `remote.log.delete.on.disable=false`?
>
> IIUC, the purpose of `remote.log.delete.on.disable` is to get explicit
> confirmation from the user
> before deleting the remote log segments. The concern raised in the thread
> is that if the user
> accidentally changes the value of `remote.storage.enable` from true to
> false, then remote segments
> get lost.
>
> For ungraceful disablement, (ie) disabling the remote storage for the
> topic and deleting all the
> remote segments, the user should set both the configs at once:
>
> (1) remote.storage.enable = false and remote.log.delete.on.disable = true
>
> If the user accidentally sets only the remote.storage.enable = true and
> leaves the `remote.log.delete.on.disable`
> with default value of `false`, then we will throw ConfigException to
> prevent the deletion of remote logs.
>
> For graceful disablement, the user should set:
>
> (1): remote.copy.disabled = true.
> (2): Wait for all the remote segments to be deleted async due to breach by
> retention time (or) size,
>then set the `remote.storage.enable = false`. This step is
> optional.
>
> Luke,
>
> In ZK mode, once the topic config value gets updated, then it gets saved
> in the /configs/topics/ znode.
> If we throw an exception from the server for invalid config, then there
> will be inconsistency between the CLI tools
> and the actual state of the topic in the cluster. This can cause some
> confusion to the users whether tiered storage
> is disabled or not. I don't know how the Kraft topic config
> propagation/validation works.
>
> --
> Kamal
>
> On Thu, Jul 25, 2024 at 7:10 PM Chia-Ping Tsai  wrote:
>
>> remote.storage.enable=false
>> remote.log.delete.on.disable=false (default)
>> If the topic config is set to this, or changed to this, we'll return
>> ConfigException during validation.
>>
>> Pardon me, I'm a bit confused.
>>
>> when `remote.storage.enable=true`, `remote.log.delete.on.disable=false` is
>> no-op
>> when `remote.storage.enable=false`, `remote.log.delete.on.disable=false`
>> is
>> error
>>
>> If `remote.log.delete.on.disable` must be true when setting
>> `remote.storage.enable`
>> to false, does it mean changing `remote.storage.enable` to false is
>> expected to delete remote storage topic data"?
>>
>>  If so, what is the purpose of `remote.log.delete.on.disable=false`?
>>
>> Best,
>> Chia-Ping
>>
>> Luke Chen  於 2024年7月25日 週四 下午8:51寫道:
>>
>> > Hi Christo,
>> >
>> > Thanks for your reply.
>> >
>> > > keep the remote.log.disable.policy, but only allow it to take a value
>> of
>> > "delete".
>> >
>> > I agree, or maybe make it a boolean value, and rename it to
>> > `remote.log.delete.on.disable`, which is clearer.
>> > And because of this new config, there will be a case that the config is
>> > like this:
>> >
>> > remote.storage.enable=false
>> > remote.log.delete.on.disable=false (default)
>> >
>> > That means, in this case, we'll keep all remote storage data, but close
>> all
>> > remote log tasks, and make "log start offset = local log start offset".
>> > This will make the remote storage metadata in an unknown state because
>> the
>> > data in the remote storage is inaccessible anymore (since log start
>> moved
>> > to LLSO). And once this topic re-enables the `remote.storage.enable`,
>> the
>> > old remote log metadata will be included, but log start offset is not
>> > expected anymore
>> >
>> > So, I'd like to propose that we don't allow this configuration:
>> >
>> > remote.storage.enable=false
>> > remote.log.delete.on.disable=false (default)
>> >
>> > If the topic config is set to this, or changed to this, we'll return
>> > ConfigException during validation.
>> >
>> > To make it clear, this is the new proposed solution:
>> >
>> >
>> https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit
>> >
>> > Let me know what you think.
>> >
>> > Thanks.
>> > Luke
>> >
>> >
>> >
>> > On Thu, Jul 25, 2024 at 8:07 PM Christo Lolov 
>> > wrote:
>> >
>> > > Hello!
>> > >
>> > > Thank you for raising this!
>> > >
>> > > Up to now KIP-950 took the stance that you can disable tiering
>> whenever
>> > you
>> > > wish as long as you specify what you would like to do with the data in
>> > > remote. Amongst other things it also made the promise that it will not
>> > > delete data without a user explicitly saying that they want their data
>> > > deleted. In other words there is a 2-step verification that the user
>> > truly
>> > > wants their data deleted.
>> > >
>> > > From the table of the new proposal I am left with the impression that
>> the
>> > > moment a user 

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Kamal Chandraprakash
Hi Chia-Ping,

Thanks for the review!

>  If so, what is the purpose of `remote.log.delete.on.disable=false`?

IIUC, the purpose of `remote.log.delete.on.disable` is to get explicit
confirmation from the user
before deleting the remote log segments. The concern raised in the thread
is that if the user
accidentally changes the value of `remote.storage.enable` from true to
false, then remote segments
get lost.

For ungraceful disablement, (ie) disabling the remote storage for the topic
and deleting all the
remote segments, the user should set both the configs at once:

(1) remote.storage.enable = false and remote.log.delete.on.disable = true

If the user accidentally sets only the remote.storage.enable = true and
leaves the `remote.log.delete.on.disable`
with default value of `false`, then we will throw ConfigException to
prevent the deletion of remote logs.

For graceful disablement, the user should set:

(1): remote.copy.disabled = true.
(2): Wait for all the remote segments to be deleted async due to breach by
retention time (or) size,
   then set the `remote.storage.enable = false`. This step is optional.

Luke,

In ZK mode, once the topic config value gets updated, then it gets saved in
the /configs/topics/ znode.
If we throw an exception from the server for invalid config, then there
will be inconsistency between the CLI tools
and the actual state of the topic in the cluster. This can cause some
confusion to the users whether tiered storage
is disabled or not. I don't know how the Kraft topic config
propagation/validation works.

--
Kamal

On Thu, Jul 25, 2024 at 7:10 PM Chia-Ping Tsai  wrote:

> remote.storage.enable=false
> remote.log.delete.on.disable=false (default)
> If the topic config is set to this, or changed to this, we'll return
> ConfigException during validation.
>
> Pardon me, I'm a bit confused.
>
> when `remote.storage.enable=true`, `remote.log.delete.on.disable=false` is
> no-op
> when `remote.storage.enable=false`, `remote.log.delete.on.disable=false` is
> error
>
> If `remote.log.delete.on.disable` must be true when setting
> `remote.storage.enable`
> to false, does it mean changing `remote.storage.enable` to false is
> expected to delete remote storage topic data"?
>
>  If so, what is the purpose of `remote.log.delete.on.disable=false`?
>
> Best,
> Chia-Ping
>
> Luke Chen  於 2024年7月25日 週四 下午8:51寫道:
>
> > Hi Christo,
> >
> > Thanks for your reply.
> >
> > > keep the remote.log.disable.policy, but only allow it to take a value
> of
> > "delete".
> >
> > I agree, or maybe make it a boolean value, and rename it to
> > `remote.log.delete.on.disable`, which is clearer.
> > And because of this new config, there will be a case that the config is
> > like this:
> >
> > remote.storage.enable=false
> > remote.log.delete.on.disable=false (default)
> >
> > That means, in this case, we'll keep all remote storage data, but close
> all
> > remote log tasks, and make "log start offset = local log start offset".
> > This will make the remote storage metadata in an unknown state because
> the
> > data in the remote storage is inaccessible anymore (since log start moved
> > to LLSO). And once this topic re-enables the `remote.storage.enable`, the
> > old remote log metadata will be included, but log start offset is not
> > expected anymore
> >
> > So, I'd like to propose that we don't allow this configuration:
> >
> > remote.storage.enable=false
> > remote.log.delete.on.disable=false (default)
> >
> > If the topic config is set to this, or changed to this, we'll return
> > ConfigException during validation.
> >
> > To make it clear, this is the new proposed solution:
> >
> >
> https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit
> >
> > Let me know what you think.
> >
> > Thanks.
> > Luke
> >
> >
> >
> > On Thu, Jul 25, 2024 at 8:07 PM Christo Lolov 
> > wrote:
> >
> > > Hello!
> > >
> > > Thank you for raising this!
> > >
> > > Up to now KIP-950 took the stance that you can disable tiering whenever
> > you
> > > wish as long as you specify what you would like to do with the data in
> > > remote. Amongst other things it also made the promise that it will not
> > > delete data without a user explicitly saying that they want their data
> > > deleted. In other words there is a 2-step verification that the user
> > truly
> > > wants their data deleted.
> > >
> > > From the table of the new proposal I am left with the impression that
> the
> > > moment a user tries to disable tiering their data will by deleted. In
> > other
> > > words, there is no 2-step verification that they want their data
> deleted.
> > >
> > > On a first read, I wouldn't be opposed to this proposal since it
> > provides a
> > > neat alternative to the tiered epoch as long as there is still a 2-step
> > > verification that the user is aware their data will be deleted. I think
> > > that a reasonable way to achieve this is to keep the
> > > remote.log.disable.policy, but only allow it to take a 

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Chia-Ping Tsai
remote.storage.enable=false
remote.log.delete.on.disable=false (default)
If the topic config is set to this, or changed to this, we'll return
ConfigException during validation.

Pardon me, I'm a bit confused.

when `remote.storage.enable=true`, `remote.log.delete.on.disable=false` is
no-op
when `remote.storage.enable=false`, `remote.log.delete.on.disable=false` is
error

If `remote.log.delete.on.disable` must be true when setting
`remote.storage.enable`
to false, does it mean changing `remote.storage.enable` to false is
expected to delete remote storage topic data"?

 If so, what is the purpose of `remote.log.delete.on.disable=false`?

Best,
Chia-Ping

Luke Chen  於 2024年7月25日 週四 下午8:51寫道:

> Hi Christo,
>
> Thanks for your reply.
>
> > keep the remote.log.disable.policy, but only allow it to take a value of
> "delete".
>
> I agree, or maybe make it a boolean value, and rename it to
> `remote.log.delete.on.disable`, which is clearer.
> And because of this new config, there will be a case that the config is
> like this:
>
> remote.storage.enable=false
> remote.log.delete.on.disable=false (default)
>
> That means, in this case, we'll keep all remote storage data, but close all
> remote log tasks, and make "log start offset = local log start offset".
> This will make the remote storage metadata in an unknown state because the
> data in the remote storage is inaccessible anymore (since log start moved
> to LLSO). And once this topic re-enables the `remote.storage.enable`, the
> old remote log metadata will be included, but log start offset is not
> expected anymore
>
> So, I'd like to propose that we don't allow this configuration:
>
> remote.storage.enable=false
> remote.log.delete.on.disable=false (default)
>
> If the topic config is set to this, or changed to this, we'll return
> ConfigException during validation.
>
> To make it clear, this is the new proposed solution:
>
> https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit
>
> Let me know what you think.
>
> Thanks.
> Luke
>
>
>
> On Thu, Jul 25, 2024 at 8:07 PM Christo Lolov 
> wrote:
>
> > Hello!
> >
> > Thank you for raising this!
> >
> > Up to now KIP-950 took the stance that you can disable tiering whenever
> you
> > wish as long as you specify what you would like to do with the data in
> > remote. Amongst other things it also made the promise that it will not
> > delete data without a user explicitly saying that they want their data
> > deleted. In other words there is a 2-step verification that the user
> truly
> > wants their data deleted.
> >
> > From the table of the new proposal I am left with the impression that the
> > moment a user tries to disable tiering their data will by deleted. In
> other
> > words, there is no 2-step verification that they want their data deleted.
> >
> > On a first read, I wouldn't be opposed to this proposal since it
> provides a
> > neat alternative to the tiered epoch as long as there is still a 2-step
> > verification that the user is aware their data will be deleted. I think
> > that a reasonable way to achieve this is to keep the
> > remote.log.disable.policy, but only allow it to take a value of "delete".
> >
> > What are your thoughts?
> >
> > Best,
> > Christo
> >
> >
> > On Thu, 25 Jul 2024 at 12:10, Luke Chen  wrote:
> >
> > > Hi all,
> > >
> > > I just found the table is not able to be displayed correctly in the
> > email.
> > > I've put the table content in google doc here
> > > <
> > >
> >
> https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit?usp=sharing
> > > >
> > > .
> > >
> > > Thanks.
> > > Luke
> > >
> > > On Thu, Jul 25, 2024 at 6:30 PM Luke Chen  wrote:
> > >
> > > > Hi all,
> > > >
> > > > While implementing the feature in KRaft mode, I found something we
> need
> > > to
> > > > change the original proposal:
> > > >
> > > > (1) In the KIP of "Disablement - KRaft backed Cluster
> > > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement#KIP950:TieredStorageDisablement-Disablement-KRaftBackedCluster
> > > >",
> > > > we said:
> > > > Controller persists configuration change and completes disablement:
> > > >
> > > >1. The controller creates a ConfigRecord and persists it in the
> > > >metadata topic.
> > > >2. The controller creates a TopicRecord to increment the
> > tiered_epoch
> > > >and update the tiered_state to DISABLED state.
> > > >3. This update marks the completion of the disablement process,
> > > >indicating that tiered storage has been successfully disabled for
> > the
> > > >KRaft-backed clusters. Similar to topic deletion all replicas will
> > > >eventually pick up the changes from the cluster metadata topic and
> > > apply
> > > >them to their own state. Any deletion failures will be picked up
> by
> > > the
> > > >expiration threads which should be deleting data before the log
> > start
> > > >offset. If the retention 

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Luke Chen
Hi Christo,

Thanks for your reply.

> keep the remote.log.disable.policy, but only allow it to take a value of
"delete".

I agree, or maybe make it a boolean value, and rename it to
`remote.log.delete.on.disable`, which is clearer.
And because of this new config, there will be a case that the config is
like this:

remote.storage.enable=false
remote.log.delete.on.disable=false (default)

That means, in this case, we'll keep all remote storage data, but close all
remote log tasks, and make "log start offset = local log start offset".
This will make the remote storage metadata in an unknown state because the
data in the remote storage is inaccessible anymore (since log start moved
to LLSO). And once this topic re-enables the `remote.storage.enable`, the
old remote log metadata will be included, but log start offset is not
expected anymore

So, I'd like to propose that we don't allow this configuration:

remote.storage.enable=false
remote.log.delete.on.disable=false (default)

If the topic config is set to this, or changed to this, we'll return
ConfigException during validation.

To make it clear, this is the new proposed solution:
https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit

Let me know what you think.

Thanks.
Luke



On Thu, Jul 25, 2024 at 8:07 PM Christo Lolov 
wrote:

> Hello!
>
> Thank you for raising this!
>
> Up to now KIP-950 took the stance that you can disable tiering whenever you
> wish as long as you specify what you would like to do with the data in
> remote. Amongst other things it also made the promise that it will not
> delete data without a user explicitly saying that they want their data
> deleted. In other words there is a 2-step verification that the user truly
> wants their data deleted.
>
> From the table of the new proposal I am left with the impression that the
> moment a user tries to disable tiering their data will by deleted. In other
> words, there is no 2-step verification that they want their data deleted.
>
> On a first read, I wouldn't be opposed to this proposal since it provides a
> neat alternative to the tiered epoch as long as there is still a 2-step
> verification that the user is aware their data will be deleted. I think
> that a reasonable way to achieve this is to keep the
> remote.log.disable.policy, but only allow it to take a value of "delete".
>
> What are your thoughts?
>
> Best,
> Christo
>
>
> On Thu, 25 Jul 2024 at 12:10, Luke Chen  wrote:
>
> > Hi all,
> >
> > I just found the table is not able to be displayed correctly in the
> email.
> > I've put the table content in google doc here
> > <
> >
> https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit?usp=sharing
> > >
> > .
> >
> > Thanks.
> > Luke
> >
> > On Thu, Jul 25, 2024 at 6:30 PM Luke Chen  wrote:
> >
> > > Hi all,
> > >
> > > While implementing the feature in KRaft mode, I found something we need
> > to
> > > change the original proposal:
> > >
> > > (1) In the KIP of "Disablement - KRaft backed Cluster
> > > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement#KIP950:TieredStorageDisablement-Disablement-KRaftBackedCluster
> > >",
> > > we said:
> > > Controller persists configuration change and completes disablement:
> > >
> > >1. The controller creates a ConfigRecord and persists it in the
> > >metadata topic.
> > >2. The controller creates a TopicRecord to increment the
> tiered_epoch
> > >and update the tiered_state to DISABLED state.
> > >3. This update marks the completion of the disablement process,
> > >indicating that tiered storage has been successfully disabled for
> the
> > >KRaft-backed clusters. Similar to topic deletion all replicas will
> > >eventually pick up the changes from the cluster metadata topic and
> > apply
> > >them to their own state. Any deletion failures will be picked up by
> > the
> > >expiration threads which should be deleting data before the log
> start
> > >offset. If the retention policy is delete, a new expiration thread
> > will be
> > >started on leadership change on any historical tiered topic to
> > confirm that
> > >there aren't any leftover segments in remote which need deletion.
> > After a
> > >cycle in which it didn't delete anything, it will die.
> > >
> > > For the (b) step, I don't think the controller needs to create
> > TopicRecord
> > > because:
> > > 1. The broker can fetch the "tiered_state" from the ConfigRecord
> > > 2. The "tiered_epoch" is not necessary because raft protocol will keep
> > the
> > > order for us. The broker can rely on the raft protocol and apply them
> in
> > > order, to get the expected results.
> > > 3. Marking the completion of the disablement process. In KRaft, it's
> not
> > > necessary because once the ConfigRecord is accepted by the controller,
> it
> > > must be applied by all the observers "in order".
> > >
> > > So, I'd like to propose to 

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Christo Lolov
Hello!

Thank you for raising this!

Up to now KIP-950 took the stance that you can disable tiering whenever you
wish as long as you specify what you would like to do with the data in
remote. Amongst other things it also made the promise that it will not
delete data without a user explicitly saying that they want their data
deleted. In other words there is a 2-step verification that the user truly
wants their data deleted.

>From the table of the new proposal I am left with the impression that the
moment a user tries to disable tiering their data will by deleted. In other
words, there is no 2-step verification that they want their data deleted.

On a first read, I wouldn't be opposed to this proposal since it provides a
neat alternative to the tiered epoch as long as there is still a 2-step
verification that the user is aware their data will be deleted. I think
that a reasonable way to achieve this is to keep the
remote.log.disable.policy, but only allow it to take a value of "delete".

What are your thoughts?

Best,
Christo


On Thu, 25 Jul 2024 at 12:10, Luke Chen  wrote:

> Hi all,
>
> I just found the table is not able to be displayed correctly in the email.
> I've put the table content in google doc here
> <
> https://docs.google.com/document/d/1Y_cSkXr-qQiFFlFoGqfzGHE9m9MnIvZSgGpFP5l5o4I/edit?usp=sharing
> >
> .
>
> Thanks.
> Luke
>
> On Thu, Jul 25, 2024 at 6:30 PM Luke Chen  wrote:
>
> > Hi all,
> >
> > While implementing the feature in KRaft mode, I found something we need
> to
> > change the original proposal:
> >
> > (1) In the KIP of "Disablement - KRaft backed Cluster
> > <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement#KIP950:TieredStorageDisablement-Disablement-KRaftBackedCluster
> >",
> > we said:
> > Controller persists configuration change and completes disablement:
> >
> >1. The controller creates a ConfigRecord and persists it in the
> >metadata topic.
> >2. The controller creates a TopicRecord to increment the tiered_epoch
> >and update the tiered_state to DISABLED state.
> >3. This update marks the completion of the disablement process,
> >indicating that tiered storage has been successfully disabled for the
> >KRaft-backed clusters. Similar to topic deletion all replicas will
> >eventually pick up the changes from the cluster metadata topic and
> apply
> >them to their own state. Any deletion failures will be picked up by
> the
> >expiration threads which should be deleting data before the log start
> >offset. If the retention policy is delete, a new expiration thread
> will be
> >started on leadership change on any historical tiered topic to
> confirm that
> >there aren't any leftover segments in remote which need deletion.
> After a
> >cycle in which it didn't delete anything, it will die.
> >
> > For the (b) step, I don't think the controller needs to create
> TopicRecord
> > because:
> > 1. The broker can fetch the "tiered_state" from the ConfigRecord
> > 2. The "tiered_epoch" is not necessary because raft protocol will keep
> the
> > order for us. The broker can rely on the raft protocol and apply them in
> > order, to get the expected results.
> > 3. Marking the completion of the disablement process. In KRaft, it's not
> > necessary because once the ConfigRecord is accepted by the controller, it
> > must be applied by all the observers "in order".
> >
> > So, I'd like to propose to remove the (b) step in KRaft mode.
> >
> > (2) Current configuration make users and implementation confusion.
> > This is what originally we proposed in KIP-950:
> >
> > remote.storage.enable
> >
> > remote.log.disable.policy(new)
> >
> > remote storage data
> >
> > true
> >
> > null/retain/delete
> >
> > uploadable + readable
> >
> > false
> >
> > retain (default)
> >
> > readable, but remote storage is disabled? For users, they are also
> > surprised if this topic is reading data from remote storage.
> >
> > Note: This also makes development difficult because it’s unable to
> > distinguish between:
> >
> > (1) a topic never enables remote storage
> >
> > (2) a topic enabled and then disabled remote storage
> >
> > A problem we have is when broker startup and trying to set log start
> > offset. Since the remote storage is disabled, we originally should set to
> > “local log start offset”, but in case (2), we expect it to treat it as
> > “remote storage enabled”, which is confusing.
> >
> > false
> >
> > delete
> >
> > All remote data are deleted
> >
> >
> > Therefore, Kamal and I would like to propose a new version of the
> > configuration:
> >
> > remote.storage.enable
> >
> > remote.copy.disabled (new)
> >
> > remote storage data
> >
> > true
> >
> > false (default)
> >
> > uploadable + readable
> >
> > true
> >
> > true
> >
> > readable
> >
> > false
> >
> > true/false
> >
> > All remote data are deleted
> >
> > The advantage is this config makes users clear what it is configuring,
> and
> > the 

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Luke Chen
Hi all,

I just found the table is not able to be displayed correctly in the email.
I've put the table content in google doc here

.

Thanks.
Luke

On Thu, Jul 25, 2024 at 6:30 PM Luke Chen  wrote:

> Hi all,
>
> While implementing the feature in KRaft mode, I found something we need to
> change the original proposal:
>
> (1) In the KIP of "Disablement - KRaft backed Cluster
> ",
> we said:
> Controller persists configuration change and completes disablement:
>
>1. The controller creates a ConfigRecord and persists it in the
>metadata topic.
>2. The controller creates a TopicRecord to increment the tiered_epoch
>and update the tiered_state to DISABLED state.
>3. This update marks the completion of the disablement process,
>indicating that tiered storage has been successfully disabled for the
>KRaft-backed clusters. Similar to topic deletion all replicas will
>eventually pick up the changes from the cluster metadata topic and apply
>them to their own state. Any deletion failures will be picked up by the
>expiration threads which should be deleting data before the log start
>offset. If the retention policy is delete, a new expiration thread will be
>started on leadership change on any historical tiered topic to confirm that
>there aren't any leftover segments in remote which need deletion. After a
>cycle in which it didn't delete anything, it will die.
>
> For the (b) step, I don't think the controller needs to create TopicRecord
> because:
> 1. The broker can fetch the "tiered_state" from the ConfigRecord
> 2. The "tiered_epoch" is not necessary because raft protocol will keep the
> order for us. The broker can rely on the raft protocol and apply them in
> order, to get the expected results.
> 3. Marking the completion of the disablement process. In KRaft, it's not
> necessary because once the ConfigRecord is accepted by the controller, it
> must be applied by all the observers "in order".
>
> So, I'd like to propose to remove the (b) step in KRaft mode.
>
> (2) Current configuration make users and implementation confusion.
> This is what originally we proposed in KIP-950:
>
> remote.storage.enable
>
> remote.log.disable.policy(new)
>
> remote storage data
>
> true
>
> null/retain/delete
>
> uploadable + readable
>
> false
>
> retain (default)
>
> readable, but remote storage is disabled? For users, they are also
> surprised if this topic is reading data from remote storage.
>
> Note: This also makes development difficult because it’s unable to
> distinguish between:
>
> (1) a topic never enables remote storage
>
> (2) a topic enabled and then disabled remote storage
>
> A problem we have is when broker startup and trying to set log start
> offset. Since the remote storage is disabled, we originally should set to
> “local log start offset”, but in case (2), we expect it to treat it as
> “remote storage enabled”, which is confusing.
>
> false
>
> delete
>
> All remote data are deleted
>
>
> Therefore, Kamal and I would like to propose a new version of the
> configuration:
>
> remote.storage.enable
>
> remote.copy.disabled (new)
>
> remote storage data
>
> true
>
> false (default)
>
> uploadable + readable
>
> true
>
> true
>
> readable
>
> false
>
> true/false
>
> All remote data are deleted
>
> The advantage is this config makes users clear what it is configuring, and
> the result is expected.
> Also, on the implementation side, we can still rely on
> "remote.storage.enable" to identify is this feature is on/off.
>
> Any thoughts about it?
>
> Thank you.
> Luke
>
>
>
> On Thu, May 30, 2024 at 6:50 PM David Jacot 
> wrote:
>
>> Hi all,
>>
>> Thanks for the KIP. This is definitely a worthwhile feature. However, I am
>> a bit sceptical on the ZK part of the story. The 3.8 release is supposed
>> to
>> be the last one supporting ZK so I don't really see how we could bring it
>> to ZK, knowing that we don't plan to do a 3.9 release (current plan). I
>> strongly suggest clarifying this before implementing the ZK part in order
>> to avoid having new code [1] being deleted right after 3.8 is released
>> :). Personally, I agree with Chia-Ping and Mickael. We could drop the ZK
>> part.
>>
>> [1] https://github.com/apache/kafka/pull/16131
>>
>> Best,
>> David
>>
>> On Tue, May 28, 2024 at 1:31 PM Mickael Maison 
>> wrote:
>>
>> > Hi,
>> >
>> > I agree with Chia-Ping, I think we could drop the ZK variant
>> > altogether, especially if this is not going to make it in 3.8.0.
>> > Even if we end up needing a 3.9.0 release, I wouldn't write a bunch of
>> > new ZooKeeper-related code in that release to delete it all right
>> > after in 4.0.
>> >
>> > Thanks,
>> > Mickael
>> >
>> > On Fri, May 24, 2024 at 5:03 PM Christo 

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Luke Chen
Hi all,

While implementing the feature in KRaft mode, I found something we need to
change the original proposal:

(1) In the KIP of "Disablement - KRaft backed Cluster
",
we said:
Controller persists configuration change and completes disablement:

   1. The controller creates a ConfigRecord and persists it in the metadata
   topic.
   2. The controller creates a TopicRecord to increment the tiered_epoch
   and update the tiered_state to DISABLED state.
   3. This update marks the completion of the disablement process,
   indicating that tiered storage has been successfully disabled for the
   KRaft-backed clusters. Similar to topic deletion all replicas will
   eventually pick up the changes from the cluster metadata topic and apply
   them to their own state. Any deletion failures will be picked up by the
   expiration threads which should be deleting data before the log start
   offset. If the retention policy is delete, a new expiration thread will be
   started on leadership change on any historical tiered topic to confirm that
   there aren't any leftover segments in remote which need deletion. After a
   cycle in which it didn't delete anything, it will die.

For the (b) step, I don't think the controller needs to create TopicRecord
because:
1. The broker can fetch the "tiered_state" from the ConfigRecord
2. The "tiered_epoch" is not necessary because raft protocol will keep the
order for us. The broker can rely on the raft protocol and apply them in
order, to get the expected results.
3. Marking the completion of the disablement process. In KRaft, it's not
necessary because once the ConfigRecord is accepted by the controller, it
must be applied by all the observers "in order".

So, I'd like to propose to remove the (b) step in KRaft mode.

(2) Current configuration make users and implementation confusion.
This is what originally we proposed in KIP-950:

remote.storage.enable

remote.log.disable.policy(new)

remote storage data

true

null/retain/delete

uploadable + readable

false

retain (default)

readable, but remote storage is disabled? For users, they are also
surprised if this topic is reading data from remote storage.

Note: This also makes development difficult because it’s unable to
distinguish between:

(1) a topic never enables remote storage

(2) a topic enabled and then disabled remote storage

A problem we have is when broker startup and trying to set log start
offset. Since the remote storage is disabled, we originally should set to
“local log start offset”, but in case (2), we expect it to treat it as
“remote storage enabled”, which is confusing.

false

delete

All remote data are deleted


Therefore, Kamal and I would like to propose a new version of the
configuration:

remote.storage.enable

remote.copy.disabled (new)

remote storage data

true

false (default)

uploadable + readable

true

true

readable

false

true/false

All remote data are deleted

The advantage is this config makes users clear what it is configuring, and
the result is expected.
Also, on the implementation side, we can still rely on
"remote.storage.enable" to identify is this feature is on/off.

Any thoughts about it?

Thank you.
Luke



On Thu, May 30, 2024 at 6:50 PM David Jacot 
wrote:

> Hi all,
>
> Thanks for the KIP. This is definitely a worthwhile feature. However, I am
> a bit sceptical on the ZK part of the story. The 3.8 release is supposed to
> be the last one supporting ZK so I don't really see how we could bring it
> to ZK, knowing that we don't plan to do a 3.9 release (current plan). I
> strongly suggest clarifying this before implementing the ZK part in order
> to avoid having new code [1] being deleted right after 3.8 is released
> :). Personally, I agree with Chia-Ping and Mickael. We could drop the ZK
> part.
>
> [1] https://github.com/apache/kafka/pull/16131
>
> Best,
> David
>
> On Tue, May 28, 2024 at 1:31 PM Mickael Maison 
> wrote:
>
> > Hi,
> >
> > I agree with Chia-Ping, I think we could drop the ZK variant
> > altogether, especially if this is not going to make it in 3.8.0.
> > Even if we end up needing a 3.9.0 release, I wouldn't write a bunch of
> > new ZooKeeper-related code in that release to delete it all right
> > after in 4.0.
> >
> > Thanks,
> > Mickael
> >
> > On Fri, May 24, 2024 at 5:03 PM Christo Lolov 
> > wrote:
> > >
> > > Hello!
> > >
> > > I am closing this vote as ACCEPTED with 3 binding +1 (Luke, Chia-Ping
> and
> > > Satish) and 1 non-binding +1 (Kamal) - thank you for the reviews!
> > >
> > > Realistically, I don't think I have the bandwidth to get this in 3.8.0.
> > > Due to this, I will mark tentatively the Zookeeper part for 3.9 if the
> > > community decides that they do in fact want one more 3.x release.
> > > I will mark the KRaft part as ready to be started and aiming for either
> > 4.0
> > > or 3.9.

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-30 Thread David Jacot
Hi all,

Thanks for the KIP. This is definitely a worthwhile feature. However, I am
a bit sceptical on the ZK part of the story. The 3.8 release is supposed to
be the last one supporting ZK so I don't really see how we could bring it
to ZK, knowing that we don't plan to do a 3.9 release (current plan). I
strongly suggest clarifying this before implementing the ZK part in order
to avoid having new code [1] being deleted right after 3.8 is released
:). Personally, I agree with Chia-Ping and Mickael. We could drop the ZK
part.

[1] https://github.com/apache/kafka/pull/16131

Best,
David

On Tue, May 28, 2024 at 1:31 PM Mickael Maison 
wrote:

> Hi,
>
> I agree with Chia-Ping, I think we could drop the ZK variant
> altogether, especially if this is not going to make it in 3.8.0.
> Even if we end up needing a 3.9.0 release, I wouldn't write a bunch of
> new ZooKeeper-related code in that release to delete it all right
> after in 4.0.
>
> Thanks,
> Mickael
>
> On Fri, May 24, 2024 at 5:03 PM Christo Lolov 
> wrote:
> >
> > Hello!
> >
> > I am closing this vote as ACCEPTED with 3 binding +1 (Luke, Chia-Ping and
> > Satish) and 1 non-binding +1 (Kamal) - thank you for the reviews!
> >
> > Realistically, I don't think I have the bandwidth to get this in 3.8.0.
> > Due to this, I will mark tentatively the Zookeeper part for 3.9 if the
> > community decides that they do in fact want one more 3.x release.
> > I will mark the KRaft part as ready to be started and aiming for either
> 4.0
> > or 3.9.
> >
> > Best,
> > Christo
>


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-28 Thread Mickael Maison
Hi,

I agree with Chia-Ping, I think we could drop the ZK variant
altogether, especially if this is not going to make it in 3.8.0.
Even if we end up needing a 3.9.0 release, I wouldn't write a bunch of
new ZooKeeper-related code in that release to delete it all right
after in 4.0.

Thanks,
Mickael

On Fri, May 24, 2024 at 5:03 PM Christo Lolov  wrote:
>
> Hello!
>
> I am closing this vote as ACCEPTED with 3 binding +1 (Luke, Chia-Ping and
> Satish) and 1 non-binding +1 (Kamal) - thank you for the reviews!
>
> Realistically, I don't think I have the bandwidth to get this in 3.8.0.
> Due to this, I will mark tentatively the Zookeeper part for 3.9 if the
> community decides that they do in fact want one more 3.x release.
> I will mark the KRaft part as ready to be started and aiming for either 4.0
> or 3.9.
>
> Best,
> Christo


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-24 Thread Christo Lolov
Hello!

I am closing this vote as ACCEPTED with 3 binding +1 (Luke, Chia-Ping and
Satish) and 1 non-binding +1 (Kamal) - thank you for the reviews!

Realistically, I don't think I have the bandwidth to get this in 3.8.0.
Due to this, I will mark tentatively the Zookeeper part for 3.9 if the
community decides that they do in fact want one more 3.x release.
I will mark the KRaft part as ready to be started and aiming for either 4.0
or 3.9.

Best,
Christo


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-20 Thread Satish Duggana
+1
Thanks Christo for addressing the review comments. We can update the
KIP for any minor comments/clarifications.


On Thu, 16 May 2024 at 15:21, Luke Chen  wrote:
>
> Thanks Chia-Ping!
> Since ZK is going to be removed, I agree the KRaft part has higher priority.
> But if Christo or the community contributor has spare time, it's good to
> have ZK part, too!
>
> Thanks.
> Luke
>
> On Thu, May 16, 2024 at 5:45 PM Chia-Ping Tsai  wrote:
>
> > +1 but I prefer to ship it to KRaft only.
> >
> > I do concern that community have enough time to accept more feature in 3.8
> > :(
> >
> > Best,
> > Chia-Ping
> >
> > On 2024/05/14 15:20:50 Christo Lolov wrote:
> > > Heya!
> > >
> > > I would like to start a vote on KIP-950: Tiered Storage Disablement in
> > > order to catch the last Kafka release targeting Zookeeper -
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> > >
> > > Best,
> > > Christo
> > >
> >


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-16 Thread Luke Chen
Thanks Chia-Ping!
Since ZK is going to be removed, I agree the KRaft part has higher priority.
But if Christo or the community contributor has spare time, it's good to
have ZK part, too!

Thanks.
Luke

On Thu, May 16, 2024 at 5:45 PM Chia-Ping Tsai  wrote:

> +1 but I prefer to ship it to KRaft only.
>
> I do concern that community have enough time to accept more feature in 3.8
> :(
>
> Best,
> Chia-Ping
>
> On 2024/05/14 15:20:50 Christo Lolov wrote:
> > Heya!
> >
> > I would like to start a vote on KIP-950: Tiered Storage Disablement in
> > order to catch the last Kafka release targeting Zookeeper -
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> >
> > Best,
> > Christo
> >
>


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-16 Thread Chia-Ping Tsai
+1 but I prefer to ship it to KRaft only. 

I do concern that community have enough time to accept more feature in 3.8 :(

Best,
Chia-Ping

On 2024/05/14 15:20:50 Christo Lolov wrote:
> Heya!
> 
> I would like to start a vote on KIP-950: Tiered Storage Disablement in
> order to catch the last Kafka release targeting Zookeeper -
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> 
> Best,
> Christo
> 


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-15 Thread Kamal Chandraprakash
Hi Christo,

Thanks for the KIP and incorporating the review comments.
Please update the KIP with the latest details.

+1 (non-binding).

Thanks,
Kamal



On Wed, May 15, 2024 at 3:18 PM Luke Chen  wrote:

> Hi Christo,
>
> In addition to the minor comments left in the discussion thread, it LGTM.
> +1 from me.
>
> Thank you.
> Luke
>
>
> On Tue, May 14, 2024 at 11:21 PM Christo Lolov 
> wrote:
>
> > Heya!
> >
> > I would like to start a vote on KIP-950: Tiered Storage Disablement in
> > order to catch the last Kafka release targeting Zookeeper -
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> >
> > Best,
> > Christo
> >
>


Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-15 Thread Luke Chen
Hi Christo,

In addition to the minor comments left in the discussion thread, it LGTM.
+1 from me.

Thank you.
Luke


On Tue, May 14, 2024 at 11:21 PM Christo Lolov 
wrote:

> Heya!
>
> I would like to start a vote on KIP-950: Tiered Storage Disablement in
> order to catch the last Kafka release targeting Zookeeper -
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
>
> Best,
> Christo
>


[VOTE] KIP-950: Tiered Storage Disablement

2024-05-14 Thread Christo Lolov
Heya!

I would like to start a vote on KIP-950: Tiered Storage Disablement in
order to catch the last Kafka release targeting Zookeeper -
https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement

Best,
Christo