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 just relying on metadata.version?

11. Both the quorum-state file and controller.quorum.bootstrap.servers
contain endpoints. Which one takes precedence?

12. It seems that downgrading from this KIP is not supported? Could we have
a section to make it explicit?

13. controller.quorum.auto.join.enable: If this is set true, when does the
controller issue the addVoter RPC? Does it need to wait until it's caught
up? Does it issue the addVoter RPC on every restart?

14. "using the AddVoter RPC, the Admin client or the kafka-metadata-quorum
CLI.": In general, the operator doesn't know how to make RPC calls. So the
practical options are either CLI or adminClient.

15. VotersRecord: Why does it need to include name and SecurityProtocol in
EndPoints? It's meant to replace controller.quorum.voters, which only
includes host/port.

16. "The KRaft leader cannot do this for observers (brokers) since their
supported versions are not directly known by the KRaft leader."
Hmm, the KRaft leader receives BrokerRegistrationRequest that includes
supported feature versions, right?

17. UpdateVoter:
17.1 "The leader will learn the range of supported version from the
UpdateVoter RPC".
KIP-919 introduced ControllerRegistrationRequest to do that. Do we need a
new one?
17.2 Do we support changing the voter's endpoint dynamically? If not, it
seems that can be part of ControllerRegistrationRequest too.

18. AddVoter
18.1 "This RPC can be sent to a broker or controller, when sent to a
broker, the broker will forward the request to the active controller."
If it's sent to a non-active controller, it will also be forwarded to the
active controller, right?
18.2 Why do we need the name/security protocol fields in the request?
Currently, they can be derived from the configs.
    { "name": "Listeners", "type": "[]Listener", "versions": "0+",
      "about": "The endpoints that can be used to communicate with the
voter", "fields": [
      { "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
        "about": "The name of the endpoint" },
      { "name": "Host", "type": "string", "versions": "0+",
        "about": "The hostname" },
      { "name": "Port", "type": "uint16", "versions": "0+",
        "about": "The port" },
      { "name": "SecurityProtocol", "type": "int16", "versions": "0+",
        "about": "The security protocol" }
    ]}
18.3 "4. Send an ApiVersions RPC to the first listener to discover the
supported kraft.version of the new voter."
Hmm, I thought that we found using ApiVersions unreliable (
https://issues.apache.org/jira/browse/KAFKA-15230) and therefore
introduced ControllerRegistrationRequest to propagate this information.
ControllerRegistrationRequest can be made at step 1 during catchup.
18.4 "In 4., the new replica will be part of the quorum so the leader will
start sending BeginQuorumEpoch requests to this replica."
Hmm, the leader should have sent BeginQuorumEpoch at step 1 so that the new
replica can catch up from it, right? Aslo, step 4 above only mentions
ApiVersions RPC, not BeginQuorumEpoch.

19. Vote: It's kind of weird that VoterUuid is at the partition level.
VoteId and VoterUuid uniquely identify a node, right? So it seems that it
should be at the same level as VoterId?

20. EndQuorumEpoch: "PreferredSuccessor which is an array is replica ids,
will be replaced by PreferredCandidates"
20.1 But PreferredSuccessor is still listed in the response.
20.2 Why does the response need to include endPoint?

21. "The KRaft implementation and protocol describe in KIP-595 and KIP-630
never read from the log or snapshot."
This seems confusing. The KIP-595 protocol does read the log to replicate
it, right?

22. "create a snapshot at 00000000000000000000-0000000000.checkpoint with
the necessary control records (KRaftVersionRecord and VotersRecord) to make
this Kafka node the only voter for the quorum."
22.1 It seems that currently, the bootstrap checkpoint is
named bootstrap.checkpoint. Are we changing it to
00000000000000000000-0000000000.checkpoint?
22.2 Just to be clear, records in the bootstrap snapshot will be propagated
to QuorumStateData, right?

23. kafka-metadata-quorum: In the output, we include the endpoints for the
voters, but not the observers. Why the inconsistency?

24. "The tool will set the TimoutMs for the AddVoterRequest to 30 seconds."
  Earlier, we have "The TimoutMs for the AddVoter RPC will be set to 10
minutes". Should we make them consistent?

25. "The replicas will update these order list of voters set whenever the
latest snapshot id increases, a VotersRecord control record is read from
the log and the log is truncated."
  Why will the voters set change when a new snapshot is created?

26. VotersRecord includes MinSupportedVersion and MaxSupportedVersion. In
FeatureLevelRecord, we only have one finalized version. Should we be
consistent?

27. The Raft dissertation mentioned the issue of disruptive servers after
they are removed. "Once the cluster leader has created the Cnew entry, a
server that is not in Cnew will no longer receive heartbeats, so it will
time out and start new elections."
  Have we addressed this issue?

28. There are quite a few typos in the KIP.
28.1 "the voters are the replica ID and UUID is in its own voters set
  Does not read well.
28.2 "used to configured:
  used to configure
28.3 "When at voter becomes a leader"
  when a voter
28.4 "write an VotersRecord controler"
  a VotersRecord; controller
28.5 "will get bootstrap"
  bootstrapped
28.6 "the leader of the KRaft cluster metadata leader"
  the leader of the KRaft cluster metadata partition
28.7 "until the call as been acknowledge"
  has been acknowledged
28.8 "As describe in KIP-595"
  described
28.9 "The KRaft implementation and protocol describe"
  described
28.10 "In other, the KRaft topic partition won't"
  In other words
28.11 "greater than their epic"
  epoch
28.12 "so their state will be tracking using their ID and UUID"
  tracked

Thanks,

Jun

On Thu, Feb 1, 2024 at 7:27 AM Jack Vanlightly <vanligh...@apache.org>
wrote:

> 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 three voters: r1, r2, r3. Voter r3 goes offline for a
> while. In the meantime, r1 dies and gets replaced with r4, and r2 dies
> getting replaced with r5. Now the cluster is formed of r3, r4, r5. When r3
> comes back online, it tries to fetch from dead nodes and finally starts
> unending leader elections - stuck because it doesn't realise it's in a
> stale configuration whose members are all dead except for itself.
>
> Example 2
> Imagine a cluster of three voters: r1, r2, r3. Voter r3 goes offline then
> comes back and discovers the leader is r1. Again, there are many
> reconfiguration commands between its LEO and the end of the leader's log.
> It starts fetching, changing configurations as it goes until it reaches a
> stale configuration (r3, r4, r5) where it is a member but none of its peers
> are actually alive anymore. It continues to fetch from the r1, but then for
> some reason the connection to r1 is interrupted. r3 starts leader elections
> which don't get responses.
>
> Example 3
> Imagine a cluster of three voters: r1, r2, r3. Over time, many
> reconfigurations have happened and now the voters are (r4, r5, r6). The
> observer o1 starts fetching from the nodes in
> 'controller.quorum.bootstrap.servers' which includes r4. r4 responds with a
> NotLeader and that r5 is the leader. o1 starts fetching and goes through
> the motion of switching to each configuration as it learns of it in the
> log. The connection to r5 gets interrupted while it is in the configuration
> (r7, r8, r9). It attempts to fetch from these voters but none respond as
> they are all long dead, as this is a stale configuration. Does the observer
> fallback to 'controller.quorum.bootstrap.servers' for its list of voters it
> can fetch from?
>
> 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 elections (based on
> its stale configuration) which will go nowhere, until it eventually
> receives a BeginQuorumEpoch from the leader and it will learn of the leader
> and resume fetching.
>
> In the case of an observer, I suppose it must fallback to
> 'controller.quorum.voters' or  'controller.quorum.bootstrap.servers' to
> learn of the leader?
>
> Thanks
> Jack
>
>
>
> On Fri, Jan 26, 2024 at 1:36 AM José Armando García Sancio
> <jsan...@confluent.io.invalid> wrote:
>
> > Hi all,
> >
> > I have updated the KIP to include information on how KRaft controller
> > automatic joining will work.
> >
> > Thanks,
> > --
> > -José
> >
>

Reply via email to