Commit Graph

250 Commits

Author SHA1 Message Date
PoAn Yang a52aedd6ff
KAFKA-18388 test-kraft-server-start.sh should use log4j2.yaml (#18370)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 04:20:06 +08:00
kevin-wu24 9818e9db26
KAFKA-17278; Add KRaft RPC compatibility tests (#17801)
Adds RPC compatibility tests for FETCH and FETCH_SNAPSHOT.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-01-06 10:18:13 -05:00
TengYao Chi 2e4a378c27
KAFKA-18131: Improve logs for voters (#18028)
Currently, the log of LeaderState#timeUntilCheckQuorumExpires uses streams without a terminal operator, resulting in output like java.util.stream.ReferencePipeline$3@39660237.
This PR aims to fix this issue and improve the log message.

Reviewers: Luke Chen <showuon@gmail.com>
2025-01-06 18:12:46 +08:00
Ismael Juma d6f24d3665
Use `instanceof` pattern to avoid explicit cast (#18373)
This feature was introduced in Java 16.

Reviewers: David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-01-02 09:32:51 -08:00
TengYao Chi 09647c9b95
MINOR: Fix broken javadoc in BatchAccumulator (#18348)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2024-12-31 11:03:20 +00:00
Ismael Juma fe56fc98fa
KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) (#18218)
Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.

Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1  (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.

Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.

Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.

Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).

[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`

Reviewers: David Arthur <mumrah@gmail.com>
2024-12-20 19:52:00 -08:00
Alyssa Huang b73e31eb15
KAFKA-17641; Update Vote RPC with PreVote field (#17807)
Introduces v2 of Vote RPC and implements the handling of the new version of the RPC.

Many references to "candidate" in the Vote RPC are changed to the more generic "replica". Replicas sending Vote request with PreVote set to true are not candidate. They are instead prospective candidate that are attempting to become candidate.

Replicas receiving PreVote requests (vote request with PreVote=true) with an epoch equal to their own will _not_ transition to Unattached state. They will only grant the vote if they have not recently fetched from leader and the request's last epoch and offset are up-to-date with theirs.

If a replica receives a PreVote request with an epoch greater than their current epoch, they will transition to Unattached state (setting their epoch to the one from the pre-vote request) and then grant the vote if the request's last epoch and offset are up-to-date with theirs.

To avoid a possible ping-pong scenario. For example, there is 3 node quorum, leader node A disconnects from quorum, node B goes into prospective state first before node C, node B sends pre-vote request to node C still in follower state and receives back that node A is leader, node B transitions to follower while node C transitions to prospective after election timeout. If you repeat this interaction, it is possible for such replicas to transition from Follower to Prospective in perpetuity. This issue is resolved by having follower state nodes grant pre-vote requests only if they have successfully fetched from the leader at least once after becoming a follower.

This change introduces a new suite called KafkaRaftClientPreVoteTest, for additional KRaft protocol tests with respect to pre-vote.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-13 16:24:30 -05:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
Alyssa Huang e979fce94e
KAFKA-17030; Unattached voters will fetch from bootstrap servers (#17352)
Because the set of voters are dynamic (KIP-953), it is possible for a replica to believe they are a voter while the current leader doesn't have that replica in the voter set. In this replicated state, the leader will not sent BeginQuorumEpoch requests to such a replica. This means that such replicas will not be able to discover the leader.

This change will help Unattached rediscover the leader by sending Fetch requests to the the bootstrap servers.
Followers have a similar issue - if they are unable to communicate with the leader they should try contacting the bootstrap servers.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-11 11:38:14 -05:00
kevin-wu24 3a9777a667
KAFKA-14619; KRaft validate snapshot id are at batch boundaries (#17500)
When KafkaRaftClient receives a request to create a snapshot with end offset that is not aligned to a batch boundary, do not create a misaligned snapshot and instead log at info level and throw an IllegalArgumentException.

Checking that the end offset is at a batch boundary is performed by reading the log at snapshotId.offset() and checking whether the offset in question is the base offset of the returned batch.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-09 11:38:00 -05:00
yx9o 38e727fe4d
KAFKA-17864: add descriptions to fields in the agreement (#17681)
Improve descriptive information in Kafka protocol documentation.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-12-07 18:47:11 +00:00
Dongnuo Lyu e30edb3eff
KAFKA-18052: Decouple the dependency of feature stable version to the metadata version (#17886)
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).

This patch introduces a LATEST_PRODUCTION value independent from the metadata version to each feature so that the highest supported feature version can be customized by the feature owner.

The change only applies to dynamic feature upgrade. During formatting, we still use the default value associated the metadata version.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2024-12-05 11:07:47 -08:00
Mickael Maison c0a092f562
MINOR: Cleanups in raft module (#17877)
Reviewers: Yash Mayya <yash.mayya@gmail.com>
2024-11-21 15:19:07 +01:00
Justin Lee a8f84cab95
KAFKA-18001: Support UpdateRaftVoterRequest in KafkaNetworkChannel (#17773)
Adds support for UpdateRaftVoterRequest in KafkaNetworkChannel. This addresses the following scenario:

* Bootstrap a KRaft Controller quorum in dynamic mode
* Start additional controllers (as observers)
* Update kraft.version feature from 0 to 1
* Use kafka-metadata-quorum add-controller to promote an observer controller to a follower

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Alyssa Huang <ahuang@confluent.io>
2024-11-15 15:55:01 -05:00
Linsiyuan9 af53758746
KAFKA-17814 Use `final` declaration to replace the suppression `this-escape` (#17613)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-03 15:00:02 +08:00
Mahsa Seifikar dafa126223
MINOR: fix the class name of UnattachedStateWithVoteTest to match the file name (#17662)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-02 13:46:21 +08:00
Kuan-Po Tseng 60ea0999fb
KAFKA-17728 Add missing config `replica-directory-id` to raft README (#17518)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-17 11:32:27 +08:00
Logan Zhu d55b4e640c
KAFKA-17763 Remove Utils.covariantCast and use var for type inference (#17468)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 15:22:43 +08:00
Gaurav Narula b03fe66cfe
KAFKA-17759 Remove Utils.mkSet (#17460)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 21:20:43 +08:00
Gaurav Narula 34ebccc79f
KAFKA-17751; fix pollTimeout calculation in pollFollowerAsVoter (#17434)
KAFKA-16534 introduced a change to send UpdateVoterRequest every "3 * fetchTimeoutMs" if the voter's configure endpoints are different from the endpoints persisted in the KRaft log. It also introduced a regression where if the voter nodes do not need an update then updateVoterTimer wasn't reset. This resulted in a busy-loop in KafkaRaftClient#poll method resulting in high CPU usage.

This PR modifies the conditions in pollFollowerAsVoter to reset updateVoterTimer appropriately.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-10-09 17:43:01 -04:00
Mickael Maison 5624bc7c7e
MINOR: Remove hamcrest from connect:runtime, raft and server-common (#17394)
Reviewers: David Arthur <mumrah@gmail.com>
2024-10-08 14:22:45 +02:00
José Armando García Sancio 16186eabcd
KAFKA-16927; Handle expanding leader endpoints (#17363)
When a replica restarts in the follower state it is possible for the set of leader endpoints to not match the latest set of leader endpoints. Voters will discover the latest set of leader endpoints through the BEGIN_QUORUM_EPOCH request. This means that KRaft needs to allow for the replica to transition from Follower to Follower when only the set of leader endpoints has changed.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Alyssa Huang <ahuang@confluent.io>
2024-10-04 10:51:43 -04:00
Alyssa Huang 68b9770506
KAFKA-17608, KAFKA-17604, KAFKA-16963; KRaft controller crashes when active controller is removed (#17146)
This change fixes a few issues.

KAFKA-17608; KRaft controller crashes when active controller is removed
When a control batch is committed, the quorum controller currently increases the last stable offset but fails to create a snapshot for that offset. This causes an issue if the quorum controller renounces and needs to revert to that offset (which has no snapshot present). Since the control batches are no-ops for the quorum controller, it does not need to update its offsets for control records. We skip handle commit logic for control batches.

KAFKA-17604; Describe quorum output missing added voters endpoints
Describe quorum output will miss endpoints of voters which were added via AddRaftVoter. This is due to a bug in LeaderState's updateVoterAndObserverStates which will pull replica state from observer states map (which does not include endpoints). The fix is to populate endpoints from the lastVoterSet passed into the method.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@apache.org>
2024-09-26 13:56:19 -04:00
xijiu 2637d12e2f
KAFKA-16908 Refactor `QuorumConfig` with `AbstractConfig` (#17231)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-26 04:06:42 +08:00
Alyssa Huang a9a4a52c9d
KAFKA-16963: Ducktape test for KIP-853 (#17081)
Add a ducktape system test for KIP-853 quorum reconfiguration, including adding and removing voters.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-09-06 13:44:09 -07:00
Ken Huang 4d23fe92f1
KAFKA-16928: Test all of the request and response methods in RaftUtil (#16517)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-09-03 13:13:33 -07:00
José Armando García Sancio 25819cecdb
KAFKA-17426; Check node directory id for KRaft (#17017)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-08-28 11:31:58 -07:00
José Armando García Sancio 7b6d4fbc57
KAFKA-17333; ResignedState should not notify of leader change (#16900)
When a voter fails as leader (LeaderState) the quorum-state still states that it is the leader of
the epoch. When the voter starts it never starts as leader and instead starts as resigned
(ResignedState) if it was previously a leader. This causes the KRaft client to immediately notify
the state machine (e.g QuorumController) that it is leader or active. This is incorrect for two
reasons.

One, the controller cannot be notified of leadership until it has reached the LEO. If the
controller is notified before that it will generate and append records that are not based on the
latest state.

Two, it is not practical to notify of local leadership when it is resigned since any write
operation (prepareAppend and schedulePreparedAppend) will fail with NotLeaderException while KRaft
is in the resigned state.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2024-08-20 16:24:13 -07:00
Mason Chen fb7e47f6e2
KAFKA-17169: Add EndpointsTest (#16659)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2024-08-20 15:08:28 -07:00
José Armando García Sancio ee71156295
KAFKA-17332; Controller always flush and can call resign on observers (#16907)
This change includes two improvements.

When the leader removes itself from the voters set clients of RaftClient may call resign. In those cases the leader is not in the voter set and should not throw an exception.

Controllers that are observers must flush the log on every append because leader may be trying to add them to the voter set. Leader always assume that voters flush their disk before sending a Fetch request.

Reviewers: David Arthur <mumrah@gmail.com>, Alyssa Huang <ahuang@confluent.io>
2024-08-19 20:44:23 -04:00
José Armando García Sancio 20c3e7324b
KAFKA-16842; Fix config validation and support unknown voters (#16892)
This change fixes the Kafka configuration validation to take into account the reconfiguration changes to configuration and allows KRaft observers to start with an unknown set of voters.

For the Kafka configuration validation the high-level change is that now the user only needs to specify either the controller.quorum.bootstrap.servers property or the controller.quorum.voters property. The other notable change in the configuration is that controller listeners can now be (and should be) specified in advertise.listeners property.

Because Kafka can now be configured without any voters and just the bootstrap servers. The KRaft client needs to allow for an unknown set of voters during the initial startup. This is done by adding the VoterSet#empty set of voters to the KRaftControlRecordStateMachine.

Lastly the RaftClientTestContext type is updated to support this new configuration for KRaft and a test is added to verify that observers can start and send Fetch requests when the voters are unknown.

Reviewers: David Arthur <mumrah@gmail.com>
2024-08-16 15:53:13 -04:00
TengYao Chi 81f0b13a70
KAFKA-17238 Move VoterSet and ReplicaKey from raft.internals to raft (#16775)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-16 00:24:51 +08:00
Josep Prat adaf2d390f
MINOR: Fix visibility for classes exposed outside of their scope (#16886)
These 2 classes are package protected but they are part of the public
API of public methods. To have clean APIs we should make this
consistent.

Static class ReplicaState is exposed in RaftUtil#singletonDescribeQuorumResponse method which is public.

RequestSender is implemented by a public class and it's exposed in the public constructor of AddVoterHandler.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-08-15 12:10:10 -04:00
José Armando García Sancio 0f7cd4dcde
KAFKA-17304; Make RaftClient API for writing to log explicit (#16862)
RaftClient API is changed to separate the batch accumulation (RaftClient#prepareAppend) from scheduling the append of accumulated batches (RaftClient#schedulePrepatedAppend) to the KRaft log. This change is needed to better match the controller's flow of replaying the generated records before replicating them. When the controller replay records it needs to know the offset associated with the record. To compute a table offset the KafkaClient needs to be aware of the records and their log position.

The controller uses this new API by generated the cluster metadata records, compute their offset using RaftClient#prepareAppend, replay the records in the state machine, and finally allowing KRaft to append the records with RaftClient#schedulePreparedAppend.

To implement this API the BatchAccumulator is changed to also support this access pattern. This is done by adding a drainOffset to the implementation. The batch accumulator is allowed to return any record and batch that is less than the drain offset.

Lastly, this change also removes some functionality that is no longer needed like non-atomic appends and validation of the base offset.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2024-08-14 15:42:04 -04:00
Ken Huang 9a85705b56
KAFKA-17297 stabalize testHandleEndQuorumRequest (#16855)
In old code leaderId and oldLeaderId all use randomReplicaId(), thus it will be flaky when leaderId is equals oldLeaderId. When KafkaRaftClient initialize it will find leaderId and oldLeaderId, so it will tansfer to candidate, and the leaderEpoch will be + 1

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-12 07:08:47 +08:00
Alyssa Huang b4d5f163a9
KAFKA-17067; Fix KRaft transition to CandidateState (#16820)
Only voters should be able to transition to Candidate state. This removes VotedState as one of the EpochStates and moves voted information into UnattachedState.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-08-10 07:43:16 -04:00
José Armando García Sancio 8ce514a52e
KAFKA-16534; Implemeent update voter sending (#16837)
This change implements the KRaft voter sending UpdateVoter request. The
UpdateVoter RPC is used to update a voter's listeners and supported
kraft versions. The UpdateVoter RPC is sent if the replicated voter set
(VotersRecord in the log) doesn't match the local voter's supported
kraft versions and controller listeners.

To not starve the Fetch request, the UpdateVoter request is sent at most
every 3 fetch timeouts. This is required to make sure that replication
is making progress and eventually the voter set in the replicated log
matches the local voter configuration.

This change also modifies the semantic for UpdateVoter. Now the
UpdateVoter response is sent right after the leader has created the new
voter set. This is required so that updating voter can transition from
sending UpdateVoter request to sending Fetch request. If the leader
waits for the VotersRecord control record to commit before sending the
UpdateVoter response, it may never send the UpdateVoter response. This
can happen if the leader needs that voter's Fetch request to commit the
control record.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-08-08 16:16:09 -07:00
Colin Patrick McCabe 6a44fb154d
KAFKA-16523; kafka-metadata-quorum: support add-controller and remove-controller (#16774)
This PR adds support for add-controller and remove-controller in the kafka-metadata-quorum.sh
command-line tool. It also fixes some minor server-side bugs that blocked the tool from working.

In kafka-metadata-quorum.sh, the implementation of remove-controller is fairly straightforward. It
just takes some command-line flags and uses them to invoke AdminClient. The add-controller
implementation is a bit more complex because we have to look at the new controller's configuration
file. The parsing logic for the advertised.listeners and listeners server configurations that we
need was previously implemented in the :core module. However, the gradle module where
kafka-metadata-quorum.sh lives, :tools, cannot depend on :core. Therefore, I moved listener parsing
into SocketServerConfigs.listenerListToEndPoints. This will be a small step forward in our efforts
to move Kafka configuration out of :core.

I also made some minor changes in kafka-metadata-quorum.sh and Kafka-storage-tool.sh to handle
--help without displaying a backtrace on the screen, and give slightly better error messages on
stderr. Also, in DynamicVoter.toString, we now enclose the host in brackets if it contains a colon
(as IPV6 addresses can).

This PR fixes our handling of clusterId in addRaftVoter and removeRaftVoter, in two ways. Firstly,
it marks clusterId as nullable in the AddRaftVoterRequest.json and RemoveRaftVoterRequest.json
schemas, as it was always intended to be. Secondly, it allows AdminClient to optionally send
clusterId, by using AddRaftVoterOptions and RemoveRaftVoterOptions. We now also remember to
properly set timeoutMs in AddRaftVoterRequest. This PR adds unit tests for
KafkaAdminClient#addRaftVoter and KafkaAdminClient#removeRaftVoter, to make sure they are sending
the right things.

Finally, I fixed some minor server-side bugs that were blocking the handling of these RPCs.
Firstly, ApiKeys.ADD_RAFT_VOTER and ApiKeys.REMOVE_RAFT_VOTER are now marked as forwardable so that
forwarding from the broker to the active controller works correctly. Secondly,
org.apache.kafka.raft.KafkaNetworkChannel has now been updated to enable API_VERSIONS_REQUEST and
API_VERSIONS_RESPONSE.

Co-authored-by: Murali Basani muralidhar.basani@aiven.io
Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
2024-08-08 15:54:12 -07:00
Dmitry Werner c9d415f361
KAFKA-17275: Move ReplicatedCounter to test scope (#16814)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-08-08 15:56:21 +02:00
José Armando García Sancio e49c8df1f7 KAFKA-16533; Update voter handling
Add support for handling the update voter RPC. The update voter RPC is used to automatically update
the voters supported kraft versions and available endpoints as the operator upgrades and
reconfigures the KRaft controllers.

The add voter RPC is handled as follow:

1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known;
   otherwise, return the REQUEST_TIMED_OUT error.

2. Check that the cluster supports kraft.version 1; otherwise, return the UNSUPPORTED_VERSION error.

3. Check that there are no uncommitted voter changes, otherwise return the REQUEST_TIMED_OUT error.

4. Check that the updated voter still supports the currently finalized kraft.version; otherwise
   return the INVALID_REQUEST error.

5. Check that the updated voter is still listening on the default listener.

6. Append the updated VotersRecord to the log. The KRaft internal listener will read this
   uncommitted record from the log and update the voter in the set of voters.

7. Wait for the VotersRecord to commit using the majority of the voters. Return a REQUEST_TIMED_OUT
   error if it doesn't commit in time.

8. Send the UpdateVoter successful response to the voter.

This change also implements the ability for the leader to update its own entry in the voter
set when it becomes leader for an epoch. This is done by updating the voter set and writing a
control batch as the first batch in a new leader epoch.

Finally, fix a bug in KafkaAdminClient's handling of removeRaftVoterResponse where we tried to cast
the response to the wrong type.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2024-08-05 11:32:21 -07:00
Colin Patrick McCabe 02f541d4ea
KAFKA-16518: Implement KIP-853 flags for storage-tool.sh (#16669)
As part of KIP-853, storage-tool.sh now has two new flags: --standalone, and --initial-voters. This PR implements these two flags in storage-tool.sh.

There are currently two valid ways to format a cluster:

The pre-KIP-853 way, where you use a statically configured controller quorum. In this case, neither --standalone nor --initial-voters may be specified, and kraft.version must be set to 0.

The KIP-853 way, where one of --standalone and --initial-voters must be specified with the initial value of the dynamic controller quorum. In this case, kraft.version must be set to 1.

This PR moves the formatting logic out of StorageTool.scala and into Formatter.java. The tool file was never intended to get so huge, or to implement complex logic like generating metadata records. Those things should be done by code in the metadata or raft gradle modules. This is also useful for junit tests, which often need to do formatting. (The 'info' and 'random-uuid' commands remain in StorageTool.scala, for now.)

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-08-02 15:47:45 -07:00
Alyssa Huang 2cf87bff9b
KAFKA-16953; Properly implement the sending of DescribeQuorumResponse (#16637)
This change allows the KRaft leader to send the DescribeQuorumResponse version based on the schema version used by the client.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-07-29 14:36:17 -04:00
José Armando García Sancio da32dcab2c
KAKFA-16537; Implement remove voter RPC (#16670)
Implement the RemoveVoter RPC. The general algorithm is as follow:

1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known;
  otherwise return the REQUEST_TIMED_OUT error.
2. Check that the cluster supports kraft.version 1; otherwise return the UNSUPPORTED_VERSION error.
3. Check that there are no uncommitted voter changes; otherwise return the REQUEST_TIMED_OUT error.
4. Append the updated VotersRecord to the log. The KRaft internal listener will read this uncommitted
  record from the log and add the new voter to the set of voters.
5. Wait for the VotersRecord to commit using the majority of the new set of voters. Return a 
  REQUEST_TIMED_OUT error if it doesn't commit in time.
6. Send the RemoveVoter successful response to the client.
7. Resign the leadership if the leader is not in the new voter set

One thing to note is that now that KRaft supports both the remove voter and add voter RPC. Only one
change can be pending at once. This is achieved in the following ways. The AddVoter RPC checks if
there are pending AddVoter or RemoveVoter RPC. The RemoveVoter RPC checks if there are any
pending AddVoter or RemoveVoter RPC. Both RPCs check that there is no uncommitted VotersRecord.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-07-26 16:25:41 -07:00
José Armando García Sancio 9db5c2481f KAFKA-16535; Implement KRaft add voter handling
This change implements the AddVoter RPC. The high-level algorithm is as follow:

1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known;
   otherwise, return the REQUEST_TIMED_OUT error.
2. Check that the cluster supports kraft.version 1; otherwise return the UNSUPPORTED_VERSION error.
3. Check that there are no uncommitted voter changes; otherwise return the REQUEST_TIMED_OUT error.
4. Check that the new voter's id is not part of the existing voter set, otherwise return the
   DUPLICATE_VOTER error.
5. Send an API_VERSIONS RPC to the first (default) listener to discover the supported kraft.version
   of the new voter.
6. Check that the new voter supports the current kraft.version, otherwise return the
   INVALID_REQUEST error.
7. Check that the new voter is caught up to the log end offset of the leader, otherwise return a
   REQUEST_TIMED_OUT error.
8. Append the updated VotersRecord to the log. The KRaft internal listener will read this
   uncommitted record from the log and add the new voter to the set of voters.
9.  Wait for the VotersRecord to commit using the majority of the new set of voters. Return a
    REQUEST_TIMED_OUT error if it doesn't commit in time.
10. Send the AddVoter successful response to the client.

The leader implements the above algorithm by tracking 3 events: the ADD_VOTER request is received,
the API_VERSIONS response is received and finally the HWM is updated.

The state of the ADD_VOTER operation is tracked by LeaderState using the AddVoterHandlerState. The
algorithm is implemented by the AddVoterHandler type.

This change also fixes a small issue introduced by the bootstrap checkpoint (0-0.checkpoint). The
internal partition listener (KRaftControlRecordStateMachine) and the external partition listener
(KafkaRaftClient.ListenerContext) were using "nextOffset = 0" as the initial state of the reading
cursor. This was causing the bootstrap checkpoint to keep getting reloaded until the leader wrote a
record to the log. Changing the initial offset (nextOffset) to -1 allows the listeners to
distinguish between the initial state (nextOffset == -1) and the bootstrap checkpoint was loaded
but the log segment is empty (nextOffset == 0).

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-07-22 13:07:38 -07:00
Volk 43fdc6ae08
KAFKA-17122 Change the type of `clusterId` from `UUID` to `String` (#16590)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-19 10:37:58 +08:00
Mickael Maison 7e3dde99d7
MINOR: Various cleanups in raft (#16611)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-18 18:48:29 +08:00
Colin Patrick McCabe 4d3e366bc2
KAFKA-16772: Introduce kraft.version to support KIP-853 (#16230)
Introduce the KRaftVersion enum to describe the current value of kraft.version. Change a bunch of places in the code that were using raw shorts over to using this new enum.

In BrokerServer.scala, fix a bug that could cause null pointer exceptions during shutdown if we tried to shut down before fully coming up.

Do not send finalized features that are finalized as level 0, since it is a no-op.

Reviewers: dengziming <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
2024-07-16 09:31:10 -07:00
Mason Chen e1bf155270
KAFKA-17055; Use random replica ids in kraft protocol tests
All of the tests in KafkakRaftClientTest and KafkaRaftClientSnapshotTest use well known ids like 0, 1, etc. Because of this those tests were not able to catch a bug in the BeginQuorumEpoch schema were the default value for VoterId was 0 instead of -1.

Improve those tests by using random valid replica id to lower the probability that the replica id will match the default value of the schema.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-07-16 11:33:32 -04:00
Volk 15eb555b03
MINOR: Fix the typo in RaftEventSimulationTest.java and ControllerNode.java (#16591)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-15 18:15:22 +08:00
Alyssa Huang 7495e70365
KAFKA-16532; Support for first leader bootstrapping the voter set (#16518)
The first leader of a KRaft topic partition must rewrite the content of the bootstrap checkpoint (0-0.checkpoint) to the log so that it is replicated. Bootstrap checkpoints are not replicated to the followers.

The control records for KRaftVersionRecord and VotersRecord in the bootstrap checkpoint will be written in one batch along with the LeaderChangeMessage. The leader will write these control records before accepting data records from the state machine (Controller).

The leader determines that the bootstrap checkpoint has not been written to the log if the latest set of voters is located at offset -1. This is the last contained offset for the bootstrap checkpoint (0-0.checkpoint).

This change also improves the RaftClientTestContext to allow for better testing of the reconfiguration functionality. This is mainly done by allowing the voter set to be configured statically or through the bootstrap checkpoint.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Colin P. McCabe <cmccabe@apache.org>
Co-authors: José Armando García Sancio <jsancio@apache.org>
2024-07-12 13:44:21 -07:00