Jun, thanks a lot for your help. I feel that the KIP is much better
after your detailed input.
If there is no more feedback, I'll start a voting thread tomorrow
morning. I'll monitor KIP-1022's discussion thread and update this KIP
with anything that affects the KIP's specification.
Thanks,
--
-
Hi, Jose,
Thanks for the explanation. Other than depending on KIP-1022 to be
approved, the KIP looks good to me now.
Jun
On Thu, Mar 28, 2024 at 2:56 PM José Armando García Sancio
wrote:
> Hi Jun,
>
> See my comments below.
>
> On Thu, Mar 28, 2024 at 11:09 AM Jun Rao wrote:
> > If I am addin
Hi Jun,
See my comments below.
On Thu, Mar 28, 2024 at 11:09 AM Jun Rao wrote:
> If I am adding a new voter and it takes a long time (because the new voter
> is catching up), I'd want to know if the request is indeed being processed.
> I thought that's the usage of uncommitted-voter-change.
The
Hi, Jose,
Thanks for the reply.
If I am adding a new voter and it takes a long time (because the new voter
is catching up), I'd want to know if the request is indeed being processed.
I thought that's the usage of uncommitted-voter-change.
Also, I am still not sure about having multiple brokers r
Hi Jun,
On Thu, Mar 28, 2024 at 10:35 AM Jun Rao wrote:
> The following are the steps of AddVoter. The bulk of the time is probably
> in step 5, but the updated VotersRecord won't be written until step 6. So,
> ideally, the controller leader should report the pending voter as soon as
> step 1. Th
Hi, Jose,
Thanks for the reply.
The following are the steps of AddVoter. The bulk of the time is probably
in step 5, but the updated VotersRecord won't be written until step 6. So,
ideally, the controller leader should report the pending voter as soon as
step 1. The brokers and non-leader control
Hi Jun,
On Wed, Mar 27, 2024 at 2:26 PM Jun Rao wrote:
> 55.1 How does the broker and non-leader controller know the pending voters?
They are in the log. Pending voter sets are VotersRecords between the
HWM and the LEO. The leader will make sure that there is at most one
VoterRecord that is unco
Hi, Jose,
Thanks for the reply.
55.1 How does the broker and non-leader controller know the pending voters?
Jun
On Wed, Mar 27, 2024 at 1:03 PM José Armando García Sancio
wrote:
> Hi Jun,
>
> Thanks for the feedback. See my comments below.
>
> On Mon, Mar 25, 2024 at 2:21 PM Jun Rao wrote:
>
Hi Jun,
Thanks for the feedback. See my comments below.
On Mon, Mar 25, 2024 at 2:21 PM Jun Rao wrote:
> 54. Yes, we could include SecurityProtocol in DescribeQuorumResponse. Then,
> we could include it in the output of kafka-metadata-quorum --describe.
Yes, I updated the DescribeQuorumResponse
Hi, Jose,
Thanks for the reply.
54. Yes, we could include SecurityProtocol in DescribeQuorumResponse. Then,
we could include it in the output of kafka-metadata-quorum --describe.
55.1 Could number-of-observers and pending-voter-change be reported by all
brokers and controllers? I thought only th
Hi Jun,
See my comments below.
On Fri, Mar 22, 2024 at 1:30 PM Jun Rao wrote:
> 54. Admin.addMetadataVoter: It seems that Endpoint shouldn't include
> securityProtocol since it's not in DescribeQuorumResponse.
Yeah. I noticed that when I made the Admin changes. We either use a
different type in
Hi, Jose,
Thanks for the reply. A few more comments.
54. Admin.addMetadataVoter: It seems that Endpoint shouldn't include
securityProtocol since it's not in DescribeQuorumResponse.
55. Metrics:
55.1 It would be useful to be clear whether they are reported by the
controller leader, all controller
Hi Claude,
On Fri, Mar 22, 2024 at 4:36 AM Claude Warren wrote:
> Is there test code, or initial POC code for this KIP somewhere? I would like
> to help move this forward but need a few pointers to associated resources. I
> have read KIP-853 and it is beginning to sink in, but code would be n
Is there test code, or initial POC code for this KIP somewhere? I would like
to help move this forward but need a few pointers to associated resources. I
have read KIP-853 and it is beginning to sink in, but code would be nice.
Thanks,
Claude
On 2024/03/21 18:41:04 José Armando García Sancio
Hi Jun,
On Thu, Mar 14, 2024 at 3:38 PM Jun Rao wrote:
> 52. Admin client: Could you provide a bit more details on the changes?
I updated the KIP to include the API changes to the Admin client.
Thanks,
--
-José
Hi Jun,
See my comments below.
On Thu, Mar 14, 2024 at 3:38 PM Jun Rao wrote:
> 37. Have you updated the wiki? It seems that LeaderIdAndEpoch and
> NodeEpoint are still two separate structs.
It is updated now. Apologies for the delayed wiki updates but I was
dealing with other issues in the pas
Hi, Jose,
Thanks for the reply. A few more comments.
37. Have you updated the wiki? It seems that LeaderIdAndEpoch and
NodeEpoint are still two separate structs.
45. kafka-storage format --cluster-id --release-version 3.8
--standalone --config controller.properties
It seems that --release-versi
Hi Jun
Thanks for the feedback. See my comments below.
On Wed, Mar 6, 2024 at 4:47 PM Jun Rao wrote:
> 20.1. It seems that the PreferredSuccessors field didn't get fixed. It's
> still there together with PreferredCandidates.
> +{ "name": "PreferredSuccessors", "type": "[]int32", "version
Hi, Jose,
Thanks for the reply.
20.1. It seems that the PreferredSuccessors field didn't get fixed. It's
still there together with PreferredCandidates.
+{ "name": "PreferredSuccessors", "type": "[]int32", "versions":
"0",
+ "about": "A sorted list of preferred successors to start
Hi Jun,
See my comments below.
On Tue, Mar 5, 2024 at 2:57 PM Jun Rao wrote:
> 37. We don't batch multiple topic partitions in AddVoter, RemoveVoter and
> UpdateVoter requests while other requests like Vote and BeginQuorumEpoch
> support batching. Should we make them consistent?
Originally I ha
Hi, Jose,
Thanks for the reply.
30. So raft.version controls the version of Fetch among the voters. It
would be useful to document that.
36. Option 1 is fine. Could we document this in the section of
"Bootstrapping with multiple voters"?
37. We don't batch multiple topic partitions in AddVoter,
Hi Jun,
Thanks for the feedback. See my comments below.
On Fri, Mar 1, 2024 at 11:36 AM Jun Rao wrote:
> 30. Historically, we used MV to gate the version of Fetch request. Are you
> saying that voters will ignore MV and only depend on raft.version when
> choosing the version of Fetch request?
B
Hi Luke,
Thanks for the feedback. See my comments below.
I have also updated the KIP to reflect this discussion.
On Wed, Feb 28, 2024 at 4:34 AM Luke Chen wrote:
>
> > 2. After "RemoveVoter", what is the role of the node?
> > It looks like after the voter got removed from the voter set, it is n
Hi, Jose,
Thanks for the reply.
30. Historically, we used MV to gate the version of Fetch request. Are you
saying that voters will ignore MV and only depend on raft.version when
choosing the version of Fetch request?
35. Upgrading the controller listeners.
35.1 So, the protocol is that each cont
Hi Jun,
Thanks for the feedback. See my comments below.
On Tue, Feb 27, 2024 at 11:27 AM Jun Rao wrote:
> 30. Who controls RPCs like Fetch, FetchSnapshot, DescribeQuorum RPC? They
> are shared between voters and observers.
For Fetch and FetchSnapshot, this KIP adds the tagged field
ReplicaUuid
Hi Jun and Luke,
Jun,
I added a section the documents the process for upgrading the
controller listeners endpoints:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-853%3A+KRaft+Controller+Membership+Changes#KIP853:KRaftControllerMembershipChanges-Upgradingcontrollerlistener
Jun and Luke,
> 2. After "RemoveVoter", what is the role of the node?
> It looks like after the voter got removed from the voter set, it is not a
> voter anymore. But I think it can still fetch with the leader. So it
should
> be an observer, with a "process.role=controller"? And if the node was
> originally "pro
Hi, Jose,
Thanks for the reply.
30. Who controls RPCs like Fetch, FetchSnapshot, DescribeQuorum RPC? They
are shared between voters and observers.
"The admin client will send the
AddVoter and RemoveVoter requests if they are reported by ApiVersions."
Does the client use supported ApiKeys or kra
Hi Jun,
Thanks for the comments. See my comments below.
On Mon, Feb 26, 2024 at 4:48 PM Jun Rao wrote:
> 15.1 It would be useful to document the process of changing the controller
> listener. For example, what configs need to be changed in what order and
> how a voter/observer selects the listen
Hi, Jose,
Thanks for the reply.
15.1 It would be useful to document the process of changing the controller
listener. For example, what configs need to be changed in what order and
how a voter/observer selects the listener when multiple listeners are
returned.
15.2 Thinking a bit more. The primar
Hi Jun, see my comments below.
I need to make some changes to the KIP to reflect our conversation.
On Fri, Feb 23, 2024 at 1:39 PM Jun Rao wrote:
> 15.1 "In this configuration, the local replica needs to use
> CONTROLLER_PLAINTEXT because that is what is supported by all of the
> voters."
>H
Hi, Jose,
Thanks for the reply.
15. VotersRecord: Ok. I got the need for including the listener name in the
endpoint. Currently, controller.quorum.voters can only specify one endpoint
per voter. So, we can only support one listener. It's fine if we want to
extend that in this KIP.
15.1 "In this c
Jun, I updated one of the rejected ideas to better explain why KRaft
can't rely on information stored by the metadata layer.
Thanks,
--
-José
Thanks for the additional feedback Jun. Comments below.
On Fri, Feb 16, 2024 at 4:09 PM Jun Rao wrote:
> 10. "The controller state machine will instead push the brokers'
> kraft.version information to the KRaft client". If we do that, why do we
> need KRaftVersionRecord?
I am doing this as a rea
Thanks for the reply, Jose.
10. "The controller state machine will instead push the brokers'
kraft.version information to the KRaft client". If we do that, why do we
need KRaftVersionRecord?
15. Hmm, I thought controller.listener.names already provides the listener
name. It's a list so that we co
Hi Jason and Jun,
I have updated the KIP to incorporate your feedback. Some of the changes are:
1. Removed security protocol from the listener definition. The local
replica is going to assume that the remote replica has the same
listener name to security group mapping.
2. Reverted most of the cha
Thanks for the feedback Jason. See my comments below.
On Mon, Feb 5, 2024 at 3:11 PM Jason Gustafson
wrote:
> 1. When adding a voter, the KIP proposes to return a timeout error if the
> voter cannot catch up in time. It might be more useful to return a more
> specific error so that an operator ca
Hi Jun,
Thanks for the feedback. Excuse the delay, it took me a while to
properly address your detailed feedback. See my comments below.
I am going to update the KIP as outlined in this email. I will send
another email when I have made all of the changes.
On Fri, Feb 2, 2024 at 10:54 AM Jun Rao
I updated the KIP to include the information discussed in this thread.
I updated the sections "Reference explanation / Endpoints information"
and "Reference explanation / Leader election".
On Sun, Feb 11, 2024 at 1:06 PM José Armando García Sancio
wrote:
> Great examples. The short answer is that
Hi Jack, see my comments below.
On Thu, Feb 1, 2024 at 7:26 AM Jack Vanlightly wrote:
> After thinking it through, it occurs to me that in examples 1 and 2, the
> leader (of the latest configuration) should be sending BeginQuorumEpoch
> requests to r3 after a certain timeout? r3 can start electio
Hey Jose,
A few more questions:
1. When adding a voter, the KIP proposes to return a timeout error if the
voter cannot catch up in time. It might be more useful to return a more
specific error so that an operator can understand why the timeout occurred.
Alternatively, perhaps we could keep the ge
Hi, Jose,
Thanks for the KIP. A few comments below.
10. kraft.version: Functionality wise, this seems very similar to
metadata.version, which is to make sure that all brokers/controllers are on
a supported version before enabling a new feature. Could you explain why we
need a new one instead of j
Hi Jose,
I have a question about how voters and observers, which are far behind the
leader, catch-up when there are multiple reconfiguration commands in the
log between their position and the end of the log.
Here are some example situations that need clarification:
Example 1
Imagine a cluster of
Hi all,
I have updated the KIP to include information on how KRaft controller
automatic joining will work.
Thanks,
--
-José
Hi Jason, Colin and Luke,
I updated the KIP based on your feedback and my comments. Here is what
has changed:
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=217391519&selectedPageVersions=54&selectedPageVersions=52
Here is an outline of all of the protocol changes:
1.
Hi Team,
Greetings,
Apologies for the delay in reply as I was down with flu.
We actually reached out to you for IT/ SAP/ Oracle/ Infor / Microsoft “VOTEC IT
SERVICE PARTNERSHIP” “IT SERVICE OUTSOURCING” “ “PARTNER SERVICE
SUBCONTRACTING”
We have very attractive newly introduce re
Thanks for the feedback Luke. See my comments below:
On Wed, Jan 24, 2024 at 4:20 AM Luke Chen wrote:
> 1. About "VotersRecord":
>
> > When a KRaft voter becomes leader it will write a KRaftVersionRecord and
> VotersRecord to the log if the log or the latest snapshot doesn't contain
> any VotersR
Hi José,
Thanks for the great KIP. It took me quite a lot of time reading and
digesting!
Some questions and comments:
1. About "VotersRecord":
> When a KRaft voter becomes leader it will write a KRaftVersionRecord and
VotersRecord to the log if the log or the latest snapshot doesn't contain
any
Thanks Jason. Comments below.
On Wed, Jan 10, 2024 at 9:06 AM Jason Gustafson
wrote:
> One additional thought. It would be helpful to have an example to justify
> the need for this:
>
> > Wait for the fetch offset of the replica (ID, UUID) to catch up to the
> log end offset of the leader.
>
> It
Thanks for your feedback Jason and excuse the delayed response.
See comments below.
On Tue, Jan 9, 2024 at 5:08 PM Jason Gustafson
wrote:
>
> Hi Jose,
>
> Thanks for the KIP! A few initial questions below:
>
> 1. In the user experience section, the user is expected to provide supply
> the UUID f
Thanks for the feedback Colin. Comments below.
On Tue, Jan 9, 2024 at 4:58 PM Colin McCabe wrote:
> 1. restarting a controller with an empty storage directory
>
> The controller can contact the quorum to get the cluster ID and current MV.
> If the MV doesn't support quorum reconfiguration, it ca
Hey Jose,
One additional thought. It would be helpful to have an example to justify
the need for this:
> Wait for the fetch offset of the replica (ID, UUID) to catch up to the
log end offset of the leader.
It is helpful also to explain how this affects the AddVoter RPC. Do we wait
indefinitely?
On Tue, Jan 9, 2024, at 17:07, Jason Gustafson wrote:
> Hi Jose,
>
> Thanks for the KIP! A few initial questions below:
>
> 1. In the user experience section, the user is expected to provide supply
> the UUID for each voter. I'm assuming this is the directory.id coming from
> KIP-858. I thought it
Hi Jose,
Thanks for the KIP! A few initial questions below:
1. In the user experience section, the user is expected to provide supply
the UUID for each voter. I'm assuming this is the directory.id coming from
KIP-858. I thought it was generated by the format comand automatically? It
seems like we
Hi José,
Thanks for the revisions. I'm really excited to see this going forward for
Kafka 3.8.
One important piece of feedback that a lot of people have given me is that they
really want auto-formatting in KRaft mode. In other words, they want to start
up a process and just have it do the righ
Hi all,
KIP-853: KRaft Controller Membership Changes is ready for another
round of discussion.
There was a previous discussion thread at
https://lists.apache.org/thread/zb5l1fsqw9vj25zkmtnrk6xm7q3dkm1v
I have changed the KIP quite a bit since that discussion. The core
idea is still the same. I c
56 matches
Mail list logo