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 <j...@confluent.io.invalid> 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."
>    Hmm, my understanding is that what listener to use can be made pairwise
> between the sender and the receiver. There is no need to pick the same
> listener across all voters, right?

Yes, this is true. Thinking about it some more, there are a couple of issues.

In the current implementation, the Selector we use for the
NetworkClient only supports one ChannelBuilder and that channel
builder can only use one security protocol. Until we fix that in the
implementation the preferred (first) listener needs to be a listener
that is supported by all of the voters. I'll make sure to document
that in the Kafka documentation until we fix the implementation.

The other issue is that when the leader sends its endpoint in the
BeginQuorumEpoch it needs to send all of its endpoints. The leader
needs to do that because replicas may connect to each other using any
of the configured listeners. For example, let's assume that the voters
(controllers) have been upgraded to support the listeners
CONTROLLER_SSL and CONTROLLER_PLAINTEXT with CONTROLLER_SSL being the
preferred listener. It is possible that the observers (brokers) have
yet to be reconfigured to use CONTROLLER_SSL and are instead using
CONTROLLER_PLAINTEXT. That means that the voters need to know all of
the leader's listeners because observers discover the leader by
sending a Fetch request to a random endpoint in
controller.quorum.bootstrap.servers using the security protocol of the
first controller listener. I'll make that change to the KIP.

> 15.2  When bootstrapping with just one controller, we could register a
> single voter with multiple listeners from controller.listener.names and
> listeners. However, when bootstrapping with multiple controllers,
> kafka-storage only has the option to specify one listener
> (--controller-quorum-voters <replica-id>-<replica-uuid>@<host>:<port>) per
> voter. Should we make them consistent?

This is true. I was trying to avoid this but this configuration can
get quite complicated. The user will have a better experience if they
can express it as a JSON file. The configuration should just be a
VotersRecord but in JSON. Let me update the KIP.

> 18.3 : "It will not use the metadata layer (QuorumController) to update and
> persist the finalized kraft.version."
> So, we depend on KRaftVersionRecord to propagate finalized kraft.version to
> brokers/controllers? It would be useful to document that.

Yes. I'll make that clear in the KIP.

> 27. "My plan is to rely on KIP-996: Pre-vote:".
> Hmm, Diego's paper says "Unfortunately, the Pre-Vote phase does not solve
> the problem of disruptive servers". Is that a real issue?

An active leader gets disrupted if a replica increases their epoch
past the active leader's epoch.

Diego's Raft allows the replica to increase their epoch and become a
candidate once it wins a pre-vote. Diego defines pre-vote as follows:
"A candidate would first ask other servers whether its log was
up-to-date enough to get their vote. Only if the candidate believed it
could get votes from a majority of the cluster would it increment its
term and start a normal election.

Unfortunately, the Pre-Vote phase does not solve the problem of
disruptive servers: there are situations where the disruptive server’s
log is sufficiently up-to-date, but starting an election would still
be disruptive."

In other words, Diego's Raft only compares log lengths during the
pre-vote phase. That is why Diego then mentions:
"We modify the RequestVote RPC to achieve this: if a server receives a
RequestVote request within the minimum election timeout of hearing
from a current leader, it does not update its term or grant its vote."

Diego can do this for his definition of Raft because he is only
concerned with voters. There is no support for observers. Diego's Raft
nodes are either voters or they are not part of the cluster.

We cannot do that for KRaft because we have observers. When a replica
is removed from the voter set it becomes an observer. I want that
replica to be able to continue participating in the cluster. That
means that we need to stop observers from increasing their epoch even
if they think they are in the voter set.

For KRaft our definition of pre-vote is different. Voters are going to
perform both checks during the pre-vote phase. A voter rejects a Vote
request if either:
1. The sender's log is shorter than the receiver's log
2. The receiver has contacted the leader either through Fetch,
FetchSnapshot or BeginQuorumEpoch during the Fetch timeout.

Check 2 should stop an observer that thinks they are a voter from
increasing their epoch and disrupting the leader. Let me give an
example.

1. Assume a voter set of 1, 2, 3, 4 and it has been replicated to all
of the voters (1, 2, 3, 4).
2. Assume that the leader is 4 and it has been asked to remove voter 1.
3. Assume that leader 4 writes voter set 2, 3, 4 but hasn't replicated
it to the rest of the voters.
4. Assume that replica 1 hasn't been able to fetch from leader 4.

At this point replica 1 thinks that it is a voter (because it is in
the voters set 1, 2, 3, 4) and because it hasn't fetched from the
leader (4) it attempts to become leader. Before becoming a leader,
replica 1 transitions to prospective but it doesn't increase its
epoch. It sends a Vote request (with pre-vote set to true) to all of
the voters (2, 3, 4). If voters 2 and 3 only compare log lengths, they
would grant the pre-vote to replica 1. In KRaft voters 2 and 3 are
going to perform both checks: log length comparison and if they have
an active leader. Since at least one of the voters 2 and 3 has an
active leader (that is the majority in the new voters set 2, 3, 4) at
least one of them would reject the vote. That means that replica 1 can
at most get 2 votes (1 and one of 2 or 3) but the voters set for
replica 1 is (1, 2, 3, 4) so it must get at least 3 votes.

> 30. So, VotersRecords and LeaderChangeRecord are controlled by
> both kraft.version and metadata.version?

They are controlled by just the kraft.version. In the
__cluster_metadata-0 partition the control records (VotersRecords and
LeaderChangeMessage) are controlled by the kraft.version while the
"data" (metadata) records are controlled by the metadata.version.

> How do we resolve conflicts?

There shouldn't be any conflicts since they control different sets of records.

> For example, if a user downgrades kraft.version, do we automatically downgrade
> metadata.version to the matching level?

Downgrade is not supported for kraft.version but if they upgrade the
kraft.version then they are changing the set of control records (E.g.
VotersRecord and LeaderChangeMessage) that can be written to the KRaft
partition. If they change the metadata.version it changes the set of
metadata records (the JSON schemas in
metadata/src/main/resources/common/metadata/) that can be written to
the __cluster_metadata-0 topic partition.

> 31. kafka-features now has --release-software. Could we
> change kafka-storage to be consistent? Currently, kafka-storage takes
> --release-version.

Yes. I missed this. Thanks for catching the inconsistency. It should
be --release-version not --release-software.

> 32. Now that we have more than one feature, should we extend kafka-storage
> to also support --feature FEATURE like kafka-features?

Yes. I missed this. We need to support --feature in the kafka-storage
tool to allow advanced users with a different set of feature versions
than those associated with the release version.


Thanks!
--
-José

Reply via email to