Thanks Kamal, and Luke for improving the earlier solution for KRaft.

One minor comment I have is to change the config name from
"remote.log.copy.disable" to "remote.log.copy.enable" with the default
value being true.

The solution summary to disable tiered storage on a topic:

- When a user wants to disable tiered storage on a topic, we should
make sure that local.log and log.retention are same. This is to make
sure the user understands the implications of the local storage
requirements while disabling tiered storage and set them
appropriately.

- Stop copying the log segments to remote storage as broker needs to
accumulate the required data locally to serve the required data from
local storage before we disable in remote storage. This will be done
by updating the config "remote.log.copy.enable" as false.

- We added a guardrail to make sure user understands that disabling
tiered storage will delete the remote storage data. This is by setting
"remote.log.delete.on.disable" should be true before setting
"remote.storage.enable" as false.


I think it is better to refactor the KIP to have only the updated
KRaft based solution and move the ZK based solution to the appendix
for reference. wdyt?

~Satish.



On Wed, 7 Aug 2024 at 17:38, Luke Chen <show...@gmail.com> wrote:
>
> Hi all,
>
> Based on the original design:
> When tiered storage is disabled or becomes read-only on a topic, the local
> retention configuration becomes irrelevant, and all data expiration follows
> the topic-wide retention configuration exclusively.
>
> That works well. But we are afraid users will not check the document and
> "thought" the local log is bound to the local.retention.ms/bytes after
> `remote.log.copy.disable=true` (i.e. read-only remote storage). The
> confusion might cause the local disk to be full and bring down the broker.
> To avoid this "surprise" to users, we'd like to add one more validation
> when `remote.log.copy.disable` is set to true:
>   - validation: when `remote.log.copy.disable=true`,
>     -- `local.retention.ms` must equal to `retention.ms` or -2 (which
> means `retention.ms` value to be used)
>     -- `local.retention.bytes` must equal to `retention.byes` or -2 (which
> means `retention.ms` value to be used)
>
> So, basically, we don't change the original design, just want to make sure
> users are aware of the retention policy change after disabling remote log
> copy.
>
> Let me know if you have any comments.
>
> Thank you.
> Luke
>
> On Fri, Jul 26, 2024 at 8:17 PM Luke Chen <show...@gmail.com> wrote:
>
> > Thanks Kamal for the comments.
> > KIP updated.
> >
> > Thanks.
> > Luke
> >
> > On Fri, Jul 26, 2024 at 6:56 PM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> >> Luke,
> >>
> >> Thanks for confirming the topic config change validation on the controller
> >> and updating the KIP.
> >> The updated KIP LGTM.
> >>
> >> 1. Can we update the below sentence in the KIP to clarify that
> >> remote.storage.enable should be true during graceful disablement?
> >>
> >> > Users set the configuration
> >> "remote.storage.enable=false,remote.log.delete.on.disable=true", or
> >> "remote.copy.disabled=true" for the desired topic, indicating the
> >> disablement of tiered storage.
> >> to
> >> > Users set the configuration
> >> "remote.storage.enable=false,remote.log.delete.on.disable=true", or
> >> "remote.storage.enable=true,remote.copy.disabled=true" for the desired
> >> topic, indicating the disablement of tiered storage.
> >>
> >> 2. Can we clarify in the public interface that the StopReplica v5,
> >> tiered_epoch, and tiered_state changes are required only for ZK mode and
> >> won't be implemented?
> >>
> >> Thanks,
> >> Kamal
> >>
> >> On Fri, Jul 26, 2024 at 1:40 PM Luke Chen <show...@gmail.com> wrote:
> >>
> >> > Hi Kamal,
> >> >
> >> > Thanks for the comments.
> >> >
> >> > For this:
> >> > > 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.
> >> >
> >> > I've confirmed we can validate the topic configuration change on the
> >> > controller level, by comparing existing configuration and new changed
> >> > configuration.
> >> > In my local POC, we can fail the configuration change if it's invalid
> >> like
> >> > this:
> >> >
> >> > # Disable with remote.log.delete.on.disable=false (default)
> >> > bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
> >> >    --alter --entity-type topics --entity-name {topic-name} \
> >> >    --add-config 'remote.storage.enable=false'
> >> >
> >> > Error while executing config command with args '--bootstrap-server
> >> > {bootstrap-string} --entity-type topics --entity-name {topic-name}
> >> --alter
> >> > --add-config remote.storage.enable=false'
> >> > java.util.concurrent.ExecutionException:
> >> > org.apache.kafka.common.errors.InvalidConfigurationException: It is
> >> invalid
> >> > to disable remote storage without deleting remote data. If you want to
> >> keep
> >> > the remote data, but turn to read only, please set
> >> `remote.copy.disabled=
> >> > true`. If you want to disable remote storage and delete all remote data,
> >> > please set
> >> `remote.storage.enable=false,remote.log.delete.on.disable=true`.
> >> >
> >> > I've updated the KIP. Please take a look when available.
> >> >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> >> >
> >> > Thank you.
> >> > Luke
> >> >
> >> >
> >> > On Fri, Jul 26, 2024 at 2:05 AM Kamal Chandraprakash <
> >> > kamal.chandraprak...@gmail.com> wrote:
> >> >
> >> > > 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/<topic> 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 <chia7...@gmail.com>
> >> > > 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 <show...@gmail.com> 於 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 <
> >> > christolo...@gmail.com
> >> > > >
> >> > > >> > 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 <show...@gmail.com>
> >> > 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 <show...@gmail.com
> >> >
> >> > > >> 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 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
> >> > > >> > > <dja...@confluent.io.invalid
> >> > > >> > > > >
> >> > > >> > > > > 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 <
> >> > > >> > > > mickael.mai...@gmail.com>
> >> > > >> > > > >> 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 <
> >> > > >> > > christolo...@gmail.com
> >> > > >> > > > >
> >> > > >> > > > >> > 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
> >> > > >> > > > >> >
> >> > > >> > > > >>
> >> > > >> > > > >
> >> > > >> > > >
> >> > > >> > >
> >> > > >> >
> >> > > >>
> >> > > >
> >> > >
> >> >
> >>
> >

Reply via email to