Re: [DISCUSS] KIP-853: KRaft Voters Change

2024-01-29 Thread José Armando García Sancio
Hi all,

This DISCUSS thread was continue in a new thread at:

[DISCUSS] KIP-853: KRaft Controller Membership Changes:
https://lists.apache.org/thread/6o3sjvcb8dx1ozqfpltb7p0w76b4nd46

Thanks!
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Voters Change

2023-12-20 Thread Josep Prat
Hi Jose,

Thanks for the update.

Best,

Josep Prat
Open Source Engineering Director, aivenjosep.p...@aiven.io   |
+491715557497 | aiven.io
Aiven Deutschland GmbH
Alexanderufer 3-7, 10117 Berlin
Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
Amtsgericht Charlottenburg, HRB 209739 B

On Wed, Dec 20, 2023, 20:48 José Armando García Sancio
 wrote:

> Yes. I have been working on the KIP. I don't expect to change too much
> from what is currently documented. I am mainly focusing on improving
> the UX and readability of the KIP.
>
> I am hoping to be able to restart the discussion first thing next year.
>
> Thanks
>
> On Wed, Dec 20, 2023 at 6:11 AM Josep Prat 
> wrote:
> >
> > Hi Jose,
> > Did you have any time to update the KIP? Let me know if we can help in
> any
> > way.
> >
> > Thanks in advance,
> >
> > On Mon, Nov 27, 2023 at 7:04 PM José Armando García Sancio
> >  wrote:
> >
> > > On Mon, Nov 27, 2023 at 2:32 AM Josep Prat  wrote:
> > > > I wanted to revive this thread and see if there is anything
> preventing
> > > it to be voted on. Happy to help unblocking anything that might be
> holding
> > > this back.
> > >
> > > Hi Josep,
> > >
> > > Thanks for reviving the thread. I need to make some changes to the
> > > KIP. My thinking has changed a bit since I wrote this KIP. The core of
> > > the design still holds. I just want to improve the wording and
> > > usability.
> > >
> > > I should have an updated KIP this week. I'll restart the discussion
> > > thread at that point.
> > >
> > > Thanks,
> > > --
> > > -José
> > >
> >
> >
> > --
> > [image: Aiven] 
> >
> > *Josep Prat*
> > Open Source Engineering Director, *Aiven*
> > josep.p...@aiven.io   |   +491715557497
> > aiven.io    |   <
> https://www.facebook.com/aivencloud>
> >      <
> https://twitter.com/aiven_io>
> > *Aiven Deutschland GmbH*
> > Alexanderufer 3-7, 10117 Berlin
> > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> > Amtsgericht Charlottenburg, HRB 209739 B
>
>
>
> --
> -José
>


Re: [DISCUSS] KIP-853: KRaft Voters Change

2023-12-20 Thread José Armando García Sancio
Yes. I have been working on the KIP. I don't expect to change too much
from what is currently documented. I am mainly focusing on improving
the UX and readability of the KIP.

I am hoping to be able to restart the discussion first thing next year.

Thanks

On Wed, Dec 20, 2023 at 6:11 AM Josep Prat  wrote:
>
> Hi Jose,
> Did you have any time to update the KIP? Let me know if we can help in any
> way.
>
> Thanks in advance,
>
> On Mon, Nov 27, 2023 at 7:04 PM José Armando García Sancio
>  wrote:
>
> > On Mon, Nov 27, 2023 at 2:32 AM Josep Prat  wrote:
> > > I wanted to revive this thread and see if there is anything preventing
> > it to be voted on. Happy to help unblocking anything that might be holding
> > this back.
> >
> > Hi Josep,
> >
> > Thanks for reviving the thread. I need to make some changes to the
> > KIP. My thinking has changed a bit since I wrote this KIP. The core of
> > the design still holds. I just want to improve the wording and
> > usability.
> >
> > I should have an updated KIP this week. I'll restart the discussion
> > thread at that point.
> >
> > Thanks,
> > --
> > -José
> >
>
>
> --
> [image: Aiven] 
>
> *Josep Prat*
> Open Source Engineering Director, *Aiven*
> josep.p...@aiven.io   |   +491715557497
> aiven.io    |   
>      
> *Aiven Deutschland GmbH*
> Alexanderufer 3-7, 10117 Berlin
> Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> Amtsgericht Charlottenburg, HRB 209739 B



-- 
-José


Re: [DISCUSS] KIP-853: KRaft Voters Change

2023-12-20 Thread Josep Prat
Hi Jose,
Did you have any time to update the KIP? Let me know if we can help in any
way.

Thanks in advance,

On Mon, Nov 27, 2023 at 7:04 PM José Armando García Sancio
 wrote:

> On Mon, Nov 27, 2023 at 2:32 AM Josep Prat  wrote:
> > I wanted to revive this thread and see if there is anything preventing
> it to be voted on. Happy to help unblocking anything that might be holding
> this back.
>
> Hi Josep,
>
> Thanks for reviving the thread. I need to make some changes to the
> KIP. My thinking has changed a bit since I wrote this KIP. The core of
> the design still holds. I just want to improve the wording and
> usability.
>
> I should have an updated KIP this week. I'll restart the discussion
> thread at that point.
>
> Thanks,
> --
> -José
>


-- 
[image: Aiven] 

*Josep Prat*
Open Source Engineering Director, *Aiven*
josep.p...@aiven.io   |   +491715557497
aiven.io    |   
     
*Aiven Deutschland GmbH*
Alexanderufer 3-7, 10117 Berlin
Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
Amtsgericht Charlottenburg, HRB 209739 B


Re: [DISCUSS] KIP-853: KRaft Voters Change

2023-11-27 Thread José Armando García Sancio
On Mon, Nov 27, 2023 at 2:32 AM Josep Prat  wrote:
> I wanted to revive this thread and see if there is anything preventing it to 
> be voted on. Happy to help unblocking anything that might be holding this 
> back.

Hi Josep,

Thanks for reviving the thread. I need to make some changes to the
KIP. My thinking has changed a bit since I wrote this KIP. The core of
the design still holds. I just want to improve the wording and
usability.

I should have an updated KIP this week. I'll restart the discussion
thread at that point.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Voters Change

2023-11-27 Thread Josep Prat
Hi community,

I wanted to revive this thread and see if there is anything preventing it to be 
voted on. Happy to help unblocking anything that might be holding this back.

Best,
Josep Prat
On 2022/07/27 15:08:24 José Armando García Sancio wrote:
> Hi all,
> 
> Community members Jason Gustafson, Colin P. McCabe and I have been
> having some offline conversations.
> 
> At a high-level KIP-853 solves the problems:
> 1) How can KRaft detect and recover from disk failures on the minority
> of the voters?
> 2) How can KRaft support a changing set of voter nodes?
> 
> I think that problem 2) is a superset of problem 1). The mechanism for
> solving problem 2) can be used to solve problem 1). This is the reason
> that I decided to design them together and proposed this KIP. Problem
> 2) adds the additional requirement of how observers (Brokers and new
> Controllers) discover the leader? KIP-853 solves this problem by
> returning the endpoint of the leader in all of the KRaft RPCs. There
> are some concerns with this approach.
> 
> To solve problem 1) we don't need to return the leader's endpoint
> since it is expressed in the controller.quorum.voters property. To
> make faster progress on 1) I have decided to create "KIP-856: KRaft
> Disk Failure Recovery" that just addresses this problem. I will be
> starting a discussion thread for KIP-856 soon.
> 
> We can continue the discussion of KIP-853 here. If KIP-856 gets
> approved I will either:
> 3) Modify KIP-853 to just describe the improvement needed on top of KIP-856.
> 4) Create a new KIP and abandon KIP-853. This new KIP will take into
> account all of the discussion from this thread.
> 
> Thanks!
> -- 
> -José
> 


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-27 Thread José Armando García Sancio
Hi all,

Community members Jason Gustafson, Colin P. McCabe and I have been
having some offline conversations.

At a high-level KIP-853 solves the problems:
1) How can KRaft detect and recover from disk failures on the minority
of the voters?
2) How can KRaft support a changing set of voter nodes?

I think that problem 2) is a superset of problem 1). The mechanism for
solving problem 2) can be used to solve problem 1). This is the reason
that I decided to design them together and proposed this KIP. Problem
2) adds the additional requirement of how observers (Brokers and new
Controllers) discover the leader? KIP-853 solves this problem by
returning the endpoint of the leader in all of the KRaft RPCs. There
are some concerns with this approach.

To solve problem 1) we don't need to return the leader's endpoint
since it is expressed in the controller.quorum.voters property. To
make faster progress on 1) I have decided to create "KIP-856: KRaft
Disk Failure Recovery" that just addresses this problem. I will be
starting a discussion thread for KIP-856 soon.

We can continue the discussion of KIP-853 here. If KIP-856 gets
approved I will either:
3) Modify KIP-853 to just describe the improvement needed on top of KIP-856.
4) Create a new KIP and abandon KIP-853. This new KIP will take into
account all of the discussion from this thread.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-27 Thread Jack Vanlightly
Hi Jose,

It's looking good!

> I think that when a replica has caught up is an implementation detail
> and we can have this detailed discussion in Jira or the PR. What do
> you think?

Yes, that sounds fine. For what it's worth, making the leader take the
decision of when an observer is caught-up or not greatly simplifies it
as the leader has all the information necessary.

Thanks
Jack


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-25 Thread José Armando García Sancio
Thanks for the feedback Colin. I updated the KIP with your suggestions
and replied to your comments below.

Colin McCabe wrote:
> 1. It seems like the proposal is to have a UUID per partition directory on 
> the voter. If I understand correctly, this is sometimes referred to as 
> "VoterUUID" and sometimes as "ReplicaUUID." The latter seems more accurate, 
> since a single voter could have multiple of these IDs, in a situation where 
> we had multiple Raft topics. So it would be good to standardize on that. 
> Also, I didn't see a description of how this would be stored in the log 
> directory. That would be good to add.

Conceptually, these are all replica UUID. In the protocol (Kafka
message) we use different field names depending on the context. This
KIP and KIP-595 uses the prefixes "Candidate", "Replica", "Voter" and
"Voted" depending on the context. I went through the KIP and made some
changes to make this consistent. Diff:
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=217391519&selectedPageVersions=24&selectedPageVersions=23

> 2. When we originally did the Raft and Quorum Controller KIPs, one 
> contentious topic was node IDs. We eventually settled on the idea that broker 
> and controller IDs were in the same ID space. So you can't (for example) have 
> a broker 3 that is in a separate JVM from controller 3. This is pretty easy 
> to enforce with a static configuration, but it seems like it will be harder 
> to do dynamically.
>
> I would like to keep this invariant. This probably requires us to reject 
> attempts to add a new quorum voter which duplicates a broker ID (except in 
> the special case of co-location!) Similarly, we should reject broker 
> registrations that duplicate an unrelated controller ID. The broker's 
> incarnation ID is the key to doing this, I think. But that requires us to 
> send the incarnation ID in many of these RPCs.
>

I think there are two cases that we want to protect.
1. The admin attempts to add a voter that is a broker using the
AddVoter RPC. The KIP protects against this case by only having
controllers (or replicas that support being voters) send the
ReplicaUuid to the cluster metadata leader. Since the AddVoter RPC
requires both ID and UUID to be specified, brokers cannot be added as
voters. I thought about adding another field to the Fetch and
FetchSnapshot request to communicate to the leader if the sending
replica supports becoming a voter but decided against it to not
increase the already large number of fields in the Fetch request.

2. The admin starts a broker that has the same ID as a voter. Kafka
currently protects against this by validating the broker configuration
and fails the validation of the node is a broker and has an id that is
enumerated in controller.quorum.voters. When using dynamic
configuration the controller.quorum.voters configuration will be
empty. In the implementation we can still protect against this case by
passing enough information to the KafkaRaftClient. For example,if we
pass the boolean "canBeVoter" to the KRaft implement then check that
its replica id is not in the voter set. If it is then it will shut
down.

> 3. Is it really necessary to put the endpoint information into the 
> AddVoterRecord? It seems like that could be figured out at runtime, like we 
> do today. If we do need it, it seems particularly weird for it to be 
> per-partition (will we have a separate TCP port for each Raft partition?) I 
> also don't know why we'd want multiple endpoints. We have that for the broker 
> because the endpoints have different uses, but that isn't the case here.

Today, voters/controllers use the hosts and ports specified in
controller.quorum.voters to establish a leader. Brokers use the hosts
and ports in controller.quorum.voters to discover the leaders. The
requirements are the following:
1. Voters/controllers need to know the endpoint of all of the voters.
This is required because at a minimum the voters need to send a Vote
request to the majority of the voters to establish leadership.
2. Brokers need to discover the leader from a list of nodes.

The endpoints in AddVoterRecord are so that the voters can discover
each other. The configuration controllers.quorum.bootstrap.servers is
so that observers (brokers and new voters) which are not required to
have replicated the voters and AddVoterRecord can discover the leader.

I noticed that I forgot to add endpoint information in all of the RPCs
that return the current leader. This is needed because it is not
guaranteed that the replica will have an AddVoterRecord for the leader
id returned in the response.

> The original rationale for multiple endpoints on the controllers was to 
> support migration from PLAINTEXT to SSL (or whatever). But that only requires 
> multiple listeners to be active on the receive side, not send side. A single 
> voter never needs more than one endpoint to contact a peer.

I agree. I removed the array ([]) and made it a single en

Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-22 Thread José Armando García Sancio
Jack Vanlightly wrote:
> - Regarding the removal of voters, when a leader appends a
> RemoveVoterRecord to its log, it immediately switches to the new
> configuration. There are two cases here:
> 1. The voter being removed is the leader itself. The KIP documents that
> the followers will continue to fetch from the leader despite these
> followers learning that the leader has been removed from the voter set.
> This is correct. It might be worth stating that the reason is that else the
> cluster can get blocked from making further progress as the followers must
> keep fetching from the leader else the leader cannot commit the record and
> resign.

Yes. The KIP now reads:
To allow this operation to be committed and for the leader to resign
the followers will continue to fetch from the leader even if the
leader is not part of the new voter set. In KRaft leader election is
triggered when the voter hasn't received a successful response in the
fetch timeout.

> 2. The voter being removed is not the leader. We should document that
> the leader will accept fetches with replicas ids (i.e. not observer
> fetches) from followers who are not in the voter set of the leader. This
> will occur because the voter being removed won't have received the
> RemoveVoteRecord yet and it must be allowed to reach this record so that:

In KRaft an observer is any replica with an (ID, UUID) or without an
(ID, UUID) that is not part of the voter set. I added a Key Terms
section with the definition of important terms that are used in the
KRaft implementation and KIPs. I'll add more terms to that section as
the discussion continues.

> - Regarding what happens when a voter leaves the voter set. When a
> non-leader voter reaches a RemoveVoterRecord where it is the subject of the
> removal, does it switch to being an observer and continue fetching? When a
> leader is removed, it carries on until it has committed the record or an
> election occurs electing a different leader. Until that point, it is
> leader but not a voter, so that makes it an observer? After it has
> committed the record and resigns (or gets deposed by an election) does it
> then start fetching as an observer?

Yeah. Good points. I was missing Handling sections for the
AddVoterRecord and RemoveVoterRecord sections. I added those sections
and they go into this detail. I should point out that observer is not
technically a state in KRaft. The better way to think about it is that
the voter set determines which states, specifically the candidate
state, a follower is allowed to transition to.

> - I think the Add/RemoveVoterRecords should also include the current voter
> set. This will make it easier to code against and also make
> troubleshooting easier. Else voter membership can only be reliably
> calculated by replaying the whole log.

Yeah. The reality of the implementation is that the replicas will have
to read the entire snapshot and log before they can determine the
voter set. I have concerns that by adding this field it will
unnecessarily complicate the snapshotting logic since it will have to
remember which AddVoterRecords were already appended to the snapshot.

I think we can make the topic partition snapshot and log more
debuggable by improving the kafka-metadata-shell. It is not part of
this KIP but I hope to write a future KIP that describes how the
kafka-metadata-shell displays information about the cluster metadata
partition.

> - Regarding the adding of voters:
> 1. We should document the problem of adding a new voter which then
> causes all progress to be blocked until the voter catches up with the
> leader. For example, in a 3 node cluster, we lose 1 node. We add a new node
> which means we have a majority = 3, with only 3 functioning nodes. Until
> the new node has caught up, the high watermark cannot advance. This can be
> solved by ensuring that to add a node we start it first as an observer and
> once it has caught up, send the AddVoter RPC to the leader. This leads to
> the question of how an observer determines that it has caught up.

Yes. I have the following in AddVoter Handling section:


--- Start of Section RPCs/AddVoter/Handling from KIP ---
When the leader receives an AddVoter request it will do the following:

1. Wait for the fetch offset of the replica (ID, UUID) to catch up to
the log end offset of the leader.
2. Wait for until there are no uncommitted add or remove voter records.
3. Append the AddVoterRecord to the log.
4. The KRaft internal listener will read this record from the log and
add the voter to the voter set.
5. Wait for the AddVoterRecord to commit using the majority of new
configuration.
6. Send the AddVoter response to the client.

In 1., the leader needs to wait for the replica to catch up because
when the AddVoterRecord is appended to the log the set of voter
changes. If the added voter is far behind then it can take some time
for it to reach the HWM. During this time the leader cannot commit
data and the quorum wil

Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-22 Thread José Armando García Sancio
Thanks Niket for your feedback. I have made changes to the KIP and
replied to your comments below.


Niket Goel wrote:
> > 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.”

Updated the Replica UUID section to better describe when it will be
generated and how it will be persisted.

> > 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)

Either solution is correct but I think that the administrator would
prefer for the operation to get held until it can get processed or it
times out.

> > 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?

Updated the last sentence of that section to the following:

The KRaft leader will not perform writes from the state machine
(active controller) or client until is has written to the log an
AddVoterRecord for every replica id in the controller.quorum.voters
configuration.

>
> > 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?

This should be safe because in this configuration the log will be
empty until all of the AddVoterRecords are persisted in a RecordBatch.
RecordBatches are atomic.

>
> > 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?

These two configurations are mutually exclusive. The Kafka cluster is
expected to use one or the other. Kafka configuration validation will
fail if both are set. Kafka doesn't automatically update
configurations.

> 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?

That example assumes that the cluster is configured to use
controller.quorum.voters: "Let's assume that the cluster is configured
to use  controller.quorum.voters and the value is
1@host1:9092,2@host2:9092,3@host3:9094."

>
> > 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.

Hmm. All of the ideas and changes in the Proposed Changes section are
required and important for this feature to be correct and safe. The
Leader Election section highlights this change to the Vote RPC. The
Vote section later on in the document goes into more details.

> > 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)

I think that this requires going into how KafkaRaftClient is
implemented. I don't think we should do that in the KIP. I think that
it is better discussed during the implementation and PR review
process. The KIP and this section highlights that the implementation
needs to handle the voter set changing either because a log record was
read or because a log record was truncated.

> > This state can be discovered by a client by using the DescribeQuorum RPC, 
> > the Admin client or th

Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-22 Thread José Armando García Sancio
Tom Bentley wrote:
> Thanks for the KIP. As Justine mentioned, this KIP currently lacks a
> motivation, and nor does the JIRA provide any context. Please could you
> provide this context, otherwise it's impossible for people on this list to
> understand the problem you're trying to solve here.

Justine Olshan wrote:
> I was curious a bit more about the motivation here. That section seems to be 
> missing.

I updated the motivation section with the following text:

KIP-595 introduced KRaft topic partitions. These are partitions with
replicas that can achieve consensus on the Kafka log without relying
on the Controller or ZK. The KRaft Controllers in KIP-631 use one of
these topic partitions (called cluster metadata topic partition) to
order operations on the cluster, commit them to disk and replicate
them to other controllers and brokers.

Consensus on the cluster metadata partition was achieved by the voters
(Controllers). If the operator of a KRaft cluster wanted to make
changes to the set of voters, they would have to  shutdown all of the
controllers nodes and manually make changes to the on-disk state of
the old controllers and new controllers. If the operator wanted to
replace an existing voter because of a disk failure or general
hardware failure, they would have to make sure that the new voter node
has a superset of the previous voter's on-disk state. Both of these
solutions are manual and error prone.

This KIP describes a protocol for extending KIP-595 and KIP-630 so
that the operator can programmatically update the voter set in a way
that is safe and is available. There are two important use cases that
this KIP supports. One use case is that the operator wants to change
the number of controllers by adding or removing a controller.  The
other use case is that the operation wants to replace a controller
because of a disk or hardware failure.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-22 Thread Tom Bentley
Hi José,

Thanks for the KIP. As Justine mentioned, this KIP currently lacks a
motivation, and nor does the JIRA provide any context. Please could you
provide this context, otherwise it's impossible for people on this list to
understand the problem you're trying to solve here.

Many thanks,

Tom

On Fri, 22 Jul 2022 at 01:04, Colin McCabe  wrote:

> Hi José,
>
> Thanks for the KIP! I have not had time to fully digest it, but I had some
> initial questions:
>
> 1. It seems like the proposal is to have a UUID per partition directory on
> the voter. If I understand correctly, this is sometimes referred to as
> "VoterUUID" and sometimes as "ReplicaUUID." The latter seems more accurate,
> since a single voter could have multiple of these IDs, in a situation where
> we had multiple Raft topics. So it would be good to standardize on that.
> Also, I didn't see a description of how this would be stored in the log
> directory. That would be good to add.
>
> 2. When we originally did the Raft and Quorum Controller KIPs, one
> contentious topic was node IDs. We eventually settled on the idea that
> broker and controller IDs were in the same ID space. So you can't (for
> example) have a broker 3 that is in a separate JVM from controller 3. This
> is pretty easy to enforce with a static configuration, but it seems like it
> will be harder to do dynamically.
>
> I would like to keep this invariant. This probably requires us to reject
> attempts to add a new quorum voter which duplicates a broker ID (except in
> the special case of co-location!) Similarly, we should reject broker
> registrations that duplicate an unrelated controller ID. The broker's
> incarnation ID is the key to doing this, I think. But that requires us to
> send the incarnation ID in many of these RPCs.
>
> 3. Is it really necessary to put the endpoint information into the
> AddVoterRecord? It seems like that could be figured out at runtime, like we
> do today. If we do need it, it seems particularly weird for it to be
> per-partition (will we have a separate TCP port for each Raft partition?) I
> also don't know why we'd want multiple endpoints. We have that for the
> broker because the endpoints have different uses, but that isn't the case
> here.
>
> The original rationale for multiple endpoints on the controllers was to
> support migration from PLAINTEXT to SSL (or whatever). But that only
> requires multiple listeners to be active on the receive side, not send
> side. A single voter never needs more than one endpoint to contact a peer.
>
> Overall, I think we'd be better off keeping this as soft state rather than
> adding it to the log. Particularly if it's not in the log at all for the
> static configuration case...
>
> 4. How do you get from the static configuration situation to the dynamic
> one? Can it be done with a rolling restart? I think the answer is yes, but
> I wasn't quite sure on reading. Does a leader using the static
> configuration auto-remove voters that aren't in that static config, as well
> as auto-add? The adding behavior is spelled out, but not removing (or maybe
> I missed it).
>
> best,
> Colin
>
>
> On Thu, Jul 21, 2022, at 09:49, José Armando García Sancio 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://cwiki.apache.org/confluence/x/nyH1D
> >
> > Thanks!
> > -José
>
>


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-21 Thread Colin McCabe
Hi José,

Thanks for the KIP! I have not had time to fully digest it, but I had some 
initial questions:

1. It seems like the proposal is to have a UUID per partition directory on the 
voter. If I understand correctly, this is sometimes referred to as "VoterUUID" 
and sometimes as "ReplicaUUID." The latter seems more accurate, since a single 
voter could have multiple of these IDs, in a situation where we had multiple 
Raft topics. So it would be good to standardize on that. Also, I didn't see a 
description of how this would be stored in the log directory. That would be 
good to add.

2. When we originally did the Raft and Quorum Controller KIPs, one contentious 
topic was node IDs. We eventually settled on the idea that broker and 
controller IDs were in the same ID space. So you can't (for example) have a 
broker 3 that is in a separate JVM from controller 3. This is pretty easy to 
enforce with a static configuration, but it seems like it will be harder to do 
dynamically.

I would like to keep this invariant. This probably requires us to reject 
attempts to add a new quorum voter which duplicates a broker ID (except in the 
special case of co-location!) Similarly, we should reject broker registrations 
that duplicate an unrelated controller ID. The broker's incarnation ID is the 
key to doing this, I think. But that requires us to send the incarnation ID in 
many of these RPCs.

3. Is it really necessary to put the endpoint information into the 
AddVoterRecord? It seems like that could be figured out at runtime, like we do 
today. If we do need it, it seems particularly weird for it to be per-partition 
(will we have a separate TCP port for each Raft partition?) I also don't know 
why we'd want multiple endpoints. We have that for the broker because the 
endpoints have different uses, but that isn't the case here.

The original rationale for multiple endpoints on the controllers was to support 
migration from PLAINTEXT to SSL (or whatever). But that only requires multiple 
listeners to be active on the receive side, not send side. A single voter never 
needs more than one endpoint to contact a peer.

Overall, I think we'd be better off keeping this as soft state rather than 
adding it to the log. Particularly if it's not in the log at all for the static 
configuration case...

4. How do you get from the static configuration situation to the dynamic one? 
Can it be done with a rolling restart? I think the answer is yes, but I wasn't 
quite sure on reading. Does a leader using the static configuration auto-remove 
voters that aren't in that static config, as well as auto-add? The adding 
behavior is spelled out, but not removing (or maybe I missed it).

best,
Colin


On Thu, Jul 21, 2022, at 09:49, José Armando García Sancio 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://cwiki.apache.org/confluence/x/nyH1D
>
> Thanks!
> -José


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-21 Thread Jack Vanlightly
Hi Jose,

Thanks for getting this started. Some comments:

- Regarding the removal of voters, when a leader appends a
RemoveVoterRecord to its log, it immediately switches to the new
configuration. There are two cases here:
1. The voter being removed is the leader itself. The KIP documents that
the followers will continue to fetch from the leader despite these
followers learning that the leader has been removed from the voter set.
This is correct. It might be worth stating that the reason is that else the
cluster can get blocked from making further progress as the followers must
keep fetching from the leader else the leader cannot commit the record and
resign.
2. The voter being removed is not the leader. We should document that
the leader will accept fetches with replicas ids (i.e. not observer
fetches) from followers who are not in the voter set of the leader. This
will occur because the voter being removed won't have received the
RemoveVoteRecord yet and it must be allowed to reach this record so that:
a) the RemoveVoterRecord can be committed (this voter may actually be
required in order to reach quorum on this record) and b) the voter being
removed can switch out of voter mode.
- Regarding what happens when a voter leaves the voter set. When a
non-leader voter reaches a RemoveVoterRecord where it is the subject of the
removal, does it switch to being an observer and continue fetching? When a
leader is removed, it carries on until it has committed the record or an
election occurs electing a different leader. Until that point, it is
leader but not a voter, so that makes it an observer? After it has
committed the record and resigns (or gets deposed by an election) does it
then start fetching as an observer?
- I think the Add/RemoveVoterRecords should also include the current voter
set. This will make it easier to code against and also make
troubleshooting easier. Else voter membership can only be reliably
calculated by replaying the whole log.
- Regarding the adding of voters:
1. We should document the problem of adding a new voter which then
causes all progress to be blocked until the voter catches up with the
leader. For example, in a 3 node cluster, we lose 1 node. We add a new node
which means we have a majority = 3, with only 3 functioning nodes. Until
the new node has caught up, the high watermark cannot advance. This can be
solved by ensuring that to add a node we start it first as an observer and
once it has caught up, send the AddVoter RPC to the leader. This leads to
the question of how an observer determines that it has caught up.
2. Perhaps an administrator should send a Join RPC to the node we want
to add. It will, if it isn't already, start fetching as an observer and
automatically do the AddVoter RPC to the leader itself. This way from a
human operations point-of-view, there is only a single action to perform.
- For any of the community that understand or are interested in TLA+,
should we include the provisional TLA+ specification for this work?
https://github.com/Vanlightly/raft-tlaplus/blob/main/specifications/pull-raft/KRaftWithReconfig.tla

Thanks
Jack


On Thu, Jul 21, 2022 at 7:38 PM Justine Olshan 
wrote:

> Hey Jose -- this seems like an important KIP! And I enjoy seeing more Uuid
> usage :)
> I was curious a bit more about the motivation here. That section seems to
> be missing.
>
> Thanks for sharing the KIP!
> Justine
>
> On Thu, Jul 21, 2022 at 10:30 AM Niket Goel 
> wrote:
>
> > 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 

Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-21 Thread Justine Olshan
Hey Jose -- this seems like an important KIP! And I enjoy seeing more Uuid
usage :)
I was curious a bit more about the motivation here. That section seems to
be missing.

Thanks for sharing the KIP!
Justine

On Thu, Jul 21, 2022 at 10:30 AM Niket Goel 
wrote:

> 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
>  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=165902699300&usg=AOvVaw12sPgdPT9X6LeINEVmj-iO
> >
> > Thanks!
> > -José
>
>


Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-21 Thread Niket Goel
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 
>  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=165902699300&usg=AOvVaw12sPgdPT9X6LeINEVmj-iO
> 
> Thanks!
> -José