Hey Jose,

Thanks for the KIP. This is a good improvement and will make the KRaft 
implementation much more robust in the face of failures and generally make it 
more flexible for users.

I did a first pass through the KIP and here are some comments (some of these 
might just be a little uninformed, so feel free to direct me to supplemental 
reading):
Overall protocol safety wise, the reconfiguration operations look safe.

> This UUID will be generated once and persisted as part of the quorum state 
> for the topic partition
Do we mean that it will be generated every time the disk on the replica is 
primed (so every disk incarnation will have UUID). I think you describe it in a 
section further below. Best to pull it up to here — “the replica uuid is 
automatically generated once by the replica when persisting the quorum state 
for the first time.”

> If there are any pending voter change operations the leader will wait for 
> them to finish.
Will new requests be rejected or queued up behind the pending operation. (I am 
assuming rejected, but want to confirm)

> When this option is used the leader of the KRaft topic partition will not 
> allow the AddVoter RPC to add replica IDs that are not describe in the 
> configuration and it would not allow the RemoveVoter RPC to remove replica 
> IDs that are described in the configuration.
Bootstrapping is a little tricky I think. Is it safer/simpler to say that the 
any add/remove RPC operations are blocked until all nodes in the config are 
processed? The way it is worded above makes it seem like the leader will accept 
adds of the same node from outside. Is that the case?

> The KRaft leader will not perform writes from the state machine (active 
> controller) until is has written to the log an AddVoterRecord for every 
> replica id in the controller.quorum.voters  configuration.
Just thinking through - One of the safety requirements for the protocol is for 
a leader to commit at least one write in an epoch before doing config changes, 
right? In this special case we should be ok because the quorum has no one but 
the leader in the beginning. Is that the thought process?

> controller.quorum.bootstrap.servers vs controller.quorum.voters
I understand the use of quorum.voters, but the bootstrap.servers is not 
entirely clear to me. So in the example of starting the cluster with one voter, 
will that one voter be listed here? And when using this configuration, is the 
expectation that quorum.voters is empty, or will it eventually get populated 
with the new quorum members? e.g. further in the kip we say — “Replica 3 will 
discover the partition leader using controller.quorum.voters”; so I guess it 
will be populated?

> This check will be removed and replicas will reply to votes request when the 
> candidate is not in the voter set or the voting replica is not in the voter 
> set.
This is a major change IMO and I think it would be good if we could somehow 
highlight it in the KIP to aid a future reader.

> This also means that the KRaft implementation needs to handle this 
> uncommitted state getting truncated and reverted.
Do we need to talk about the specific behavior a little more here? I mean how 
does this affect any inflight messages with quorums moving between different 
values. (Just a brief except to why it works)

> This state can be discovered by a client by using the DescribeQuorum RPC, the 
> Admin client or the kafka-quorum.sh CLI.
The describeQuorum RPC does not respond with a list of observers today. We 
would need a section to fix that.

> The client can now decide to add replica (3, UUID3') to the set of voters 
> using the AddVoter RPC, the Admin client or the kafka-quorum.sh CLI.
Trying the understand the general thought process‚ the addition of this node 
back into the quorum will be a manually triggered operation and not something 
the node will attempt by itself?

This is a general wonderment, and feel free to ignore it:
Might be good to list some scenarios demonstrating the safety , e.g. how do we 
ensure that there is no loss of availability during an addVoter operation when 
the leader goes down. Or how a failed operation is safely removed from the log 
and reverted.

> On Jul 21, 2022, at 9:49 AM, José Armando García Sancio 
> <jsan...@confluent.io.invalid> wrote:
> 
> Hi all,
> 
> I would like to start the discussion on my design to support
> dynamically changing the set of voters in the KRaft cluster metadata
> topic partition.
> 
> KIP URL: 
> https://www.google.com/url?q=https://cwiki.apache.org/confluence/x/nyH1D&source=gmail-imap&ust=1659026993000000&usg=AOvVaw12sPgdPT9X6LeINEVmj-iO
> 
> Thanks!
> -José

Reply via email to