Commit Graph

407 Commits

Author SHA1 Message Date
TingIāu "Ting" Kì b2758f4ac6
KAFKA-16989 Use StringBuilder instead of String concatenation (#16385)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-19 09:19:32 +08:00
Colin Patrick McCabe 2fd00ce536
KAFKA-16952: Do not bump broker epoch when re-registering the same incarnation (#16333)
* KAFKA-16952: Do not bump broker epoch when re-registering the same incarnation

As part of KIP-858 (Handle JBOD broker disk failure in KRaft), we added some code that caused the
broker to re-register itself when transitioning from a MetadataVersion that did not support broker
directory IDs, to one that did. This code was necessary because otherwise the controller would not
be aware of what directories the broker held.

However, prior to this PR, the re-registration process acted exactly like a full registration. That
is, it bumped the broker epoch (which is meant to only be bumped on broker restart). This PR fixes
the code to keep the broker epoch the same if the incarnation ID is the same.

There are some other minor improvements here:

- The previous logic relied on a complicated combination of request version and previous broker
  epoch to understand if the request came from the same broker or not. This is not needed: either
  the incarnation ID is the same and it's the same process, or it is not and it isn't.

- We now log whether we're amending a registration, registering a previously unknown broker, or
  replacing a previous registration.

- Move changes to the HeartbeatManager to the end of the function, so that we will not do them if
  any validation step fails. Log4j messages are also generated at the end, for the same reason.

Reviewers: Ron Dagostino <rndgstn@gmail.com>
2024-06-18 07:03:15 -07:00
ChickenchickenLove 1a7ba667ad
MINOR improve startup log in QuorumController (#15926)
Reviewers: David Arthur <mumrah@gmail.com>
2024-06-17 11:04:12 -04:00
TingIāu "Ting" Kì 92d8d4bd1f
KAFKA-16970 Fix hash implementation of `ScramCredentialValue`, `ScramCredentialData`, and `ContextualRecord` (#16359)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-17 22:29:22 +08:00
gongxuanzhang 4e846038a6
KAFKA-10787 Apply spotless to `metadata` and `server` and `storage` module (#16297)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-16 05:28:50 +08:00
Kuan-Po (Cooper) Tseng 2e5cd0b476
MINOR: Refine javadoc in TopicsDelta TopicDelta LocalReplicaChanges (#16195)
Add more description to TopicsDelta TopicDelta LocalReplicaChanges

Reviewers: Luke Chen <showuon@gmail.com>
2024-06-14 11:22:19 +08:00
gongxuanzhang 596b945072
KAFKA-16643 Add ModifierOrder checkstyle rule (#15890)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-13 15:39:32 +08:00
Sanskar Jhajharia 226f3c57e3
MINOR: Code cleanup in metadata module (#16065)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-06-06 15:18:23 +02:00
TingIāu "Ting" Kì d652f5cf54
MINOR: Add topicIds and directoryIds to the return value of the toString method. (#16189)
Add topicIds and directoryIds to the return value of the toString method.

Reviewers: Luke Chen <showuon@gmail.com>
2024-06-05 07:52:06 +08:00
Igor Soarez 7e0caad96e
MINOR: Cleanup unused references in core (#16192)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-05 05:12:33 +08:00
Igor Soarez 16359e70d3
KAFKA-16583: Handle PartitionChangeRecord without directory IDs (#16118)
When PartitionRegistration#merge() reads a PartitionChangeRecord
from an older MetadataVersion, with a replica assignment change
and without #directories() set, it produces a direcotry assignment
of DirectoryId.UNASSIGNED. This is problematic because the MetadataVersion
may not yet support directory assignments, leading to a
UnwritableMetadataException in PartitionRegistration#toRecord.

Since the Controller always sets directories on PartitionChangeRecord
if the MetadataVersion supports it, via PartitionChangeBuilder,
there's no need for PartitionRegistration#merge() to populate
directories upon a replica assignment change.

Reviewers: Luke Chen <showuon@gmail.com>
2024-06-04 15:37:20 +01:00
David Jacot 53d592e369
MINOR: Fix type in MetadataVersion.IBP_4_0_IV0 (#16181)
This patch fixes a typo in MetadataVersion.IBP_4_0_IV0. It should be 0 not O.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>,  Chia-Ping Tsai <chia7712@gmail.com>
2024-06-03 20:48:04 -07:00
Colin Patrick McCabe 8ace33b47f
KAFKA-16757: Fix broker re-registration issues around MV 3.7-IV2 (#15945)
When upgrading from a MetadataVersion older than 3.7-IV2, we need to resend the broker registration, so that the controller can record the storage directories. The current code for doing this has several problems, however. One is that it tends to trigger even in cases where we don't actually need it. Another is that when re-registering the broker, the broker is marked as fenced.

This PR moves the handling of the re-registration case out of BrokerMetadataPublisher and into BrokerRegistrationTracker. The re-registration code there will only trigger in the case where the broker sees an existing registration for itself with no directories set.  This is much more targetted than the original code.

Additionally, in ClusterControlManager, when re-registering the same broker, we now preserve its fencing and shutdown state, rather than clearing those. (There isn't any good reason re-registering the same broker should clear these things... this was purely an oversight.) Note that we can tell the broker is "the same" because it has the same IncarnationId.

Reviewers: Gaurav Narula <gaurav_narula2@apple.com>, Igor Soarez <soarez@apple.com>
2024-06-01 23:51:39 +01:00
David Jacot ba61ff0cd9
KAFKA-16860; [1/2] Introduce group.version feature flag (#16120)
This patch introduces the `group.version` feature flag with one version:
1) Version 1 enables the new consumer group rebalance protocol (KIP-848).

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-05-31 12:48:55 -07:00
Justine Olshan 7c1bb1585f
KAFKA-16308 [2/N]: Allow unstable feature versions and rename unstable metadata config (#16130)
As per KIP-1022, we will rename the unstable metadata versions enabled config to support all feature versions.

Features is also updated to return latest production and latest testing versions of each feature.

A feature is production ready when the corresponding metadata version (bootstrapMetadataVersion) is production ready.

Adds tests for the feature usage of the unstableFeatureVersionsEnabled config

Reviewers: David Jacot <djacot@confluent.io>, Jun Rao <junrao@gmail.com>
2024-05-30 14:52:50 -07:00
Justine Olshan 5e3df22095
KAFKA-16308 [1/N]: Create FeatureVersion interface and add `--feature` flag and handling to StorageTool (#15685)
As part of KIP-1022, I have created an interface for all the new features to be used when parsing the command line arguments, doing validations, getting default versions, etc.

I've also added the --feature flag to the storage tool to show how it will be used.

Created a TestFeatureVersion to show an implementation of the interface (besides MetadataVersion which is unique) and added tests using this new test feature.

I will add the unstable config and tests in a followup.

Reviewers: David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jun Rao <junrao@apache.org>
2024-05-29 16:36:06 -07:00
Mickael Maison affe8da54c
KAFKA-7632: Support Compression Levels (KIP-390) (#15516)
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
2024-05-21 17:58:49 +02:00
Ken Huang 81e6098021
KAFKA-16797 A bit cleanup of FeatureControlManager (#15997)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-20 17:19:01 +08:00
Gaurav Narula 412b05df00
KAFKA-16789 Fix thread leak detection for event handler threads (#15984)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-19 18:21:56 +08:00
José Armando García Sancio bfe81d6229
KAFKA-16207; KRaft's internal log listener to update voter set (#15671)
Adds support for the KafkaRaftClient to read the control records KRaftVersionRecord and VotersRecord in the snapshot and log. As the control records in the KRaft partition are read, the replica's known set of voters are updated. This change also contains the necessary changes to include the control records when a snapshot is generated by the KRaft state machine.

It is important to note that this commit changes the code and the in-memory state to track the sets of voters but it doesn't change any data that is externally exposed. It doesn't change the RPCs, data stored on disk or configuration.

When the KRaft replica starts the PartitionListener reads the latest snapshot and then log segments up to the LEO, updating the in-memory state as it reads KRaftVersionRecord and VotersRecord. When the replica (leader and follower) appends to the log, the PartitionListener catches up to the new LEO. When the replica truncates the log because of a diverging epoch, the PartitionListener also truncates the in-memory state to the new LEO. When the state machine generate a new snapshot the PartitionListener trims any prefix entries that are not needed. This is all done to minimize the amount of data tracked in-memory and to make sure that it matches the state on disk.

To implement the functionality described above this commit also makes the following changes:

Adds control records for KRaftVersionRecord and VotersRecord. KRaftVersionRecord describes the finalized kraft.version supported by all of the replicas. VotersRecords describes the set of voters at a specific offset.

Changes Kafka's feature version to support 0 as the smallest valid value. This is needed because the default value for kraft.version is 0.

Refactors FileRawSnapshotWriter so that it doesn't directly call the onSnapshotFrozen callback. It adds NotifyingRawSnapshotWriter for calling such callbacks. This reorganization is needed because in this change both the KafkaMetadataLog and the KafkaRaftClient need to react to snapshots getting frozen.

Cleans up KafkaRaftClient's initialization. Removes initialize from RaftClient - this is an implementation detail that doesn't need to be exposed in the interface. Removes RaftConfig.AddressSpec and simplifies the bootstrapping of the static voter's address. The bootstrapping of the address is delayed because of tests. We should be able to simplify this further in future commits.

Update the DumpLogSegment CLI to support the new control records KRaftVersionRecord and VotersRecord.

Fix the RecordsSnapshotReader implementations so that the iterator includes control records. RecordsIterator is extended to support reading the new control records.
Improve the BatchAccumulator implementation to allow multiple control records in one control batch. This is needed so that KRaft can make sure that VotersRecord is included in the same batch as the control record (KRaftVersionRecord) that upgrades the kraft.version to 1.

Add a History interface and default implementation TreeMapHistory. This is used to track all of the sets of voters between the latest snapshot and the LEO. This is needed so that KafkaRaftClient can query for the latest set of voters and so that KafkaRaftClient can include the correct set of voters when the state machine generates a new snapshot at a given offset.

Add a builder pattern for RecordsSnapshotWriter. The new builder pattern also implements including the KRaftVersionRecord and VotersRecord control records in the snapshot as necessary. A KRaftVersionRecord should be appended if the kraft.version is greater than 0 at the snapshot's offset. Similarly, a VotersRecord should be appended to the snapshot with the latest value up to the snapshot's offset.

Reviewers: Jason Gustafson <jason@confluent.io>
2024-05-04 12:43:16 -07:00
Colin Patrick McCabe a3f2414990
KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810)
Fix a case where we could generate useless PartitionChangeRecords on metadata versions older than
3.6-IV0. This could happen in the case where we had an ISR with only one broker in it, and we were
trying to go down to a fully empty ISR. In this case, PartitionChangeBuilder would block the record
to going down to a fully empty ISR (since that is not valid in these pre-KIP-966 metadata
versions), but it would still emit the record, even though it had no effect.

Reviewers: Igor Soarez <soarez@apple.com>
2024-05-02 09:23:25 -07:00
mannoopj 31355ef8f9
KAFKA-16475: add more tests to TopicImageNodeTest (#15735)
Add more test cases to TopicImageNodeTest.java.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-04-30 14:59:00 -07:00
Luke Chen ec151c8278
KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors (#15732)
When running ZK migrating to KRaft process, we encountered an issue that the migrating is hanging and the ZkMigrationState cannot move to MIGRATION state. And it is because the pollEvent didn't retry with the retriable MigrationClientException (ZK client retriable errors) while it should. This PR fixes it and add test. And because of this, the poll event will not poll anymore, which causes the KRaftMigrationDriver hanging.

Reviewers: Luke Chen <showuon@gmail.com>, Igor Soarez<soarez@apple.com>, Akhilesh C <akhileshchg@users.noreply.github.com>
2024-04-29 17:44:47 +08:00
Mickael Maison df4ef5a621
MINOR: Various cleanups in metadata (#15806)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-26 05:50:40 +08:00
TingIāu "Ting" Kì 864744ffd4
KAFKA-16610 Replace "Map#entrySet#forEach" by "Map#forEach" (#15795)
Reviewers: Apoorv Mittal <amittal@confluent.io>, Igor Soarez <soarez@apple.com>
2024-04-25 01:52:24 +01:00
Omnia Ibrahim cfe5ab5cf2
KAFKA-15853 Move quota configs into server-common package (#15774)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:05:18 +08:00
ilyazr b254e787cb
KAFKA-16466: Log exception message for non-fault errors in QuorumController (#15701)
The generic error handler of QuorumController didn't log the exception message for non-fault errors, which includes very useful debugging info.

Reviewers: Igor Soarez <soarez@apple.com>
2024-04-23 02:01:36 +01:00
Johnny Hsu 5193eb9323
KAFKA-16475: add test for TopicImageNodeTest (#15720)
Add a unit test for TopicImageNodeTest.

Co-authored-by: Johnny Hsu <johnnyhsu@fb.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-04-16 10:20:34 -07:00
Kuan-Po (Cooper) Tseng 315cd83048
MINOR: remove redundant argument in log (#15699)
remove redundant argument in log

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-12 08:36:33 +08:00
Igor Soarez f6c9feea76
KAFKA-16297: Race condition while promoting future replica (#15557)
If a future replica doesn't get promoted, any directory reassignment sent to the controller should be reversed.

The current logic is already addressing the case when a replica hasn't yet been promoted and the controller hasn't yet acknowledged the directory reassignment. However, it doesn't cover the case where the replica does not get promoted due to a directory failure after the controller has acknowledged the reassignment but before the future replica catches up again and is promoted to main replica.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-10 17:57:05 +08:00
Calvin Liu 6de58d2731
MINOR; Missing minISR config should log a debug message (#15529)
Log a debug message when the min isr configuration is missing for a topic.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-04-06 17:41:32 -07:00
Calvin Liu 376e9e20db
KAFKA-15586: Clean shutdown detection - server side (#14706)
If the broker registers with the same broker epoch as the previous session, it is recognized as a clean shutdown. Otherwise, it is an unclean shutdown. This replica will be removed from any ELR.

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
2024-04-04 09:12:05 -04:00
Alyssa Huang 4ccbf1634a
MINOR: Metadata image test improvements (#15373)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-03-28 11:22:02 +01:00
PoAn Yang 6f8d4fe26b
KAFKA-15949: Unify metadata.version format in log and error message (#15505)
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-26 20:09:29 +08:00
Alyssa Huang 03f7b5aa3a
KAFKA-16206: Fix unnecessary topic config deletion during ZK migration (#14206)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ron Dagostino <rndgstn@gmail.com>
2024-03-21 15:38:42 +01:00
Kuan-Po (Cooper) Tseng 12a1d85362
KAFKA-12187 replace assertTrue(obj instanceof X) with assertInstanceOf (#15512)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-20 10:36:25 +08:00
Colin Patrick McCabe 2c1943d836
MINOR: remove test constructor for PartitionAssignment (#15435)
Remove the test constructor for PartitionAssignment and remove the TODO.
Also add KRaftClusterTest.testCreatePartitions to get more coverage for
createPartitions.

Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-05 12:02:19 -08:00
Luke Chen 98fb3bd304
MINOR: log error when initialLoadFuture is not done in authorizer (#14953)
Currently, when initializing StandardAuthorizer, it'll wait until all ACL loaded and complete the initialLoadFuture. So, checking logs, we'll see:

2023-12-06 14:07:50,325 INFO [StandardAuthorizer 1] Initialized with 6 acl(s). (org.apache.kafka.metadata.authorizer.StandardAuthorizerData) [kafka-1-metadata-loader-event-handler]
2023-12-06 14:07:50,325 INFO [StandardAuthorizer 1] Completed initial ACL load process. (org.apache.kafka.metadata.authorizer.StandardAuthorizerData) [kafka-1-metadata-loader-event-handler]

But then, when shutting down the node, we will also see this error:

2023-12-06 14:12:32,752 ERROR [StandardAuthorizer 1] Failed to complete initial ACL load process. (org.apache.kafka.metadata.authorizer.StandardAuthorizerData) [kafka-1-metadata-loader-event-handler]
java.util.concurrent.TimeoutException
	at kafka.server.metadata.AclPublisher.close(AclPublisher.scala:98)
	at org.apache.kafka.image.loader.MetadataLoader.closePublisher(MetadataLoader.java:568)
	at org.apache.kafka.image.loader.MetadataLoader.lambda$removeAndClosePublisher$7(MetadataLoader.java:528)
	at org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
	at org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210)
	at org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181)
	at java.base/java.lang.Thread.run(Thread.java:840)

It's confusing. And it's because we'll try to complete authorizer initialLoad, and complete the initialLoadFuture if not done. But we'll log the error no matter it's completed or not. This patch improves the logging.

Reviewers: Josep Prat <josep.prat@aiven.io>
2024-02-17 13:58:11 +08:00
Calvin Liu 756f44a3e5
KAFKA-15665: Enforce partition reassignment should complete when all target replicas are in ISR (#15359)
When completing the partition reassignment, the new ISR should have all the target replicas.

Reviewers: Justine Olshan <jolshan@confluent.io>, David Mao <dmao@confluent.io>
2024-02-16 10:27:43 -08:00
David Arthur c000b1fae2
MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-02-07 12:54:59 -08:00
David Arthur 12ce9c7f98 KAFKA-16216: Reduce batch size for initial metadata load during ZK migration
During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control this batch size, for
testing purposes.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-02-01 15:48:52 -08:00
David Arthur 16ed7357b1 KAFKA-16171: Fix ZK migration controller race #15238
This patch causes the active KRaftMigrationDriver to reload the /migration ZK state after electing
itself as the leader in ZK. This closes a race condition where the previous active controller could
make an update to /migration after the new leader was elected. The update race was not actually a
problem regarding the data since both controllers would be syncing the same state from KRaft to ZK,
but the change to the znode causes the new controller to fail on the zk version check on
/migration.

This patch also fixes a as-yet-unseen bug where the active controllers failing to elect itself via
claimControllerLeadership would not retry.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-01-29 13:51:45 -08:00
Calvin Liu 7e5ef9b509
KAFKA-15585: Implement DescribeTopicPartitions RPC on broker (#14612)
This patch implements the new DescribeTopicPartitions RPC as defined in KIP-966 (ELR). Additionally, this patch adds a broker config "max.request.partition.size.limit" which limits the number of partitions returned by the new RPC.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>, David Arthur <mumrah@gmail.com>
2024-01-24 15:16:09 -05:00
Justine Olshan e00d36b9c0
KAFKA-15468 [1/2]: Prevent transaction coordinator reloads on already loaded leaders (#15139)
This originally was #14489 which covered 2 aspects -- reloading on partition epoch changes where leader epoch did not change and reloading when leader epoch changed but we were already the leader.

I've cut out the second part of the change since the first part is much simpler.

Redefining the TopicDelta fields to better distinguish when a leader is elected (leader epoch bump) vs when a leader has isr/replica changes (partition epoch bump). There are some cases where we bump the partition epoch but not the leader epoch. We do not need to do operations that only care about the leader epoch bump. (ie -- onElect callbacks)

Reviewers: Artem Livshits <alivshits@confluent.io>, José Armando García Sancio <jsancio@apache.org>
2024-01-23 14:58:53 -08:00
David Arthur 7bf7fd99a5
KAFKA-16078: Be more consistent about getting the latest MetadataVersion
This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2024-01-17 14:59:22 -08:00
Divij Vaidya 65424ab484
MINOR: New year code cleanup - include final keyword (#15072)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Sagar Rao <sagarmeansocean@gmail.com>
2024-01-11 17:53:35 +01:00
Igor Soarez f385ef468b
KAFKA-15364: Replay BrokerRegistrationChangeRecord.logDirs (#14998)
Any directory changes must be considered when replaying
BrokerRegistrationChangeRecord. This is necessary
to persist directory failures in the cluster metadata,
which #14902 missed.

Reviewers: Omnia G.H Ibrahim <o.g.h.ibrahim@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2023-12-18 15:43:28 -05:00
David Arthur 7f763d327f
KAFKA-16007 Merge batch records during ZK migration (#15007)
To avoid creating lots of small KRaft batches during the ZK migration, this patch adds a mechanism to merge batches into sizes of at least 1000. This has the effect of reducing the number of batches sent to Raft which reduces the amount of time spent blocking.

Since migrations use metadata transactions, the batch boundaries for migrated records are not significant. Even in light of that, this implementation does not break up existing batches. It will only combine them into a larger batch to meet the minimum size.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2023-12-15 19:33:15 -05:00
Proven Provenzano b0e99b5593
KAFKA-15922: Bump MetadataVersion to support JBOD with KRaft (#14984)
Moves ELR from MetadataVersion IBP_3_7_IV3 into the new IBP_3_8_IV0 because the ELR feature was not completed before 3.7 reached feature freeze.  Leaves IBP_3_7_IV3 empty -- it is a no-op and is not reused for anything.  Adds the new MetadataVersion IBP_3_7_IV4 for the FETCH request changes from KIP-951, which were mistakenly never associated with a MetadataVersion.  Updates the LATEST_PRODUCTION MetadataVersion to IBP_3_7_IV4 to declare both KRaft JBOD and the KIP-951 changes ready for production use.

Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@apache.org>, Justine Olshan <jolshan@confluent.io>
2023-12-14 10:08:54 -05:00
Omnia Ibrahim 07490b929b
KAFKA-15365: Broker-side replica management changes (#14881)
Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Proven Provenzano <pprovenzano@confluent.io>
2023-12-11 09:34:22 -05:00
Colin Patrick McCabe 32fdb8d173
KAFKA-15956: MetadataShell must take the log directory lock when reading (#14899)
MetadataShell should take an advisory lock on the .lock file of the directory it is reading from.
Add an integration test of this functionality in MetadataShellIntegrationTest.java.

Note: in build.gradle, I had to add some dependencies on server-common's test files in order to use
MockFaultHandler, etc.

MetadataBatchLoader.java: fix a case where a log message was incorrect.  The intention was to print
the number equivalent to (offset + index).  Instead it was printing the offset, followed by the
index. So if the offset was 100 and the index was 1, 1001 would be printed rather than 101.

Co-authored-by: Igor Soarez <i@soarez.me>
Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>
2023-12-10 19:18:34 -08:00
Igor Soarez 93b6df6173
KAFKA-15364: Handle log directory failure in the Controller (#14902)
When log directories fail, the broker will send a heartbeat listing the failed directories. This
PR implements processing offline directories in the controller's broker heartbeat handling. We
update broker registrations and generate leadership/ISR changes as necessary.

Co-authored-by: Colin P. McCabe <cmccabe@apache.org>
Reviewers: Ron Dagostino <rndgstn@gmail.com>
2023-12-08 14:44:14 -08:00
Igor Soarez c515bf51f8 KAFKA-15426: Process and persist directory assignments
Handle AssignReplicasToDirs requests, persist metadata changes
with new directory assignments and possible leader elections.

Reviewers: Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-12-07 11:44:45 -08:00
Igor Soarez 32576f61ce
MINOR: always register before touch in BrokerHeartbeatManager (#14934)
BrokerHeartbeatManager should require a call to register(brokerId) before touch(brokerId)

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ron Dagostino <rndgstn@gmail.com>
2023-12-07 10:13:39 -08:00
Colin Patrick McCabe 969bc7749c
KAFKA-15980: Add the CurrentControllerId metric (#14749)
Add the CurrentControllerId metric as described in KIP-1001. This gives us an easy way to identify the current controller by looking at the metrics of any Kafka node (broker or controller).

Reviewers: David Arthur <mumrah@gmail.com>
2023-12-06 21:03:33 -08:00
Igor Soarez f467f6bb4f
KAFKA-15361: Process and persist dir info with broker registration (#14838)
Part of JBOD KIP-858, https://cwiki.apache.org/confluence/display/KAFKA/KIP-858%3A+Handle+JBOD+broker+disk+failure+in+KRaft

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
2023-12-06 16:40:43 -05:00
Colin Patrick McCabe ebae7b26b5
MINOR: fix bug where we weren't registering SnapshotEmitterMetrics (#14918)
Fix a bug where we weren't properly exposing SnapshotEmitterMetrics. Add a test.

Reviewers: David Arthur <mumrah@gmail.com>
2023-12-04 21:32:12 -08:00
Igor Soarez 6b87c85291 KAFKA-15886: Always specify directories for new partition registrations
When creating partition registrations directories must always be defined.

If creating a partition from a PartitionRecord or PartitionChangeRecord from an older version that
does not support directory assignments, then DirectoryId.MIGRATING is assumed.

If creating a new partition, or triggering a change in assignment, DirectoryId.UNASSIGNED should be
specified, unless the target broker has a single online directory registered, in which case the
replica should be assigned directly to that single directory.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-11-30 14:10:47 -08:00
Colin Patrick McCabe a94bc8d6d5
KAFKA-15922: Add a MetadataVersion for JBOD (#14860)
Assign MetadataVersion.IBP_3_7_IV2 to JBOD.

Move KIP-966 support to MetadataVersion.IBP_3_7_IV3.

Create MetadataVersion.LATEST_PRODUCTION as the latest metadata version that can be used when formatting a
new cluster, or upgrading a cluster using kafka-features.sh. This will allow us to clearly distinguish between stable
and unstable metadata versions for the first time.

Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Calvin Liu <caliu@confluent.io>, Proven Provenzano <pprovenzano@confluent.io>
2023-11-30 10:35:13 -08:00
Jason Gustafson a35e021925
MINOR: Fix flaky `MetadataLoaderTest.testNoPublishEmptyImage` (#14875)
There is a race in the assertion on `capturedImages`. Since the future is signaled first, it is still possible to see an empty list. By adding to the collection first, we can ensure the assertion will succeed.

Reviewers: Reviewers: David Jacot <djacot@confluent.io>
2023-11-30 09:50:19 -08:00
yuyli 937578be65
MINOR: Rename method sendBrokerHeartbeat #14658
Rename sendBrokerHeartbeat to sendBrokerHeartbeatToUnfenceBrokers.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-11-27 13:30:40 -08:00
Colin Patrick McCabe 209f268d6c
KAFKA-15860: ControllerRegistration must be written out to the metadata image (#14807)
The ControllerRegistration records added in KIP-919 should be written out to the metadata
image, not just the log.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2023-11-22 21:25:21 -08:00
Igor Soarez e90692246a
KAFKA-15362: Resolve offline replicas in metadata cache (#14737)
The metadata cache now considers registered log directories
and directory assignments when determining offline replicas.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>
2023-11-21 09:40:04 -08:00
Igor Soarez a03a71d7b5 KAFKA-15357: Aggregate and propagate assignments
A new AssignmentsManager accumulates, batches, and sends KIP-858
assignment events to the Controller. Assignments are sent via
AssignReplicasToDirs requests.

Move QuorumTestHarness.formatDirectories into TestUtils so it can be
used in other test contexts.

Fix a bug in ControllerRegistration.java where the wrong version of the
record was being generated in ControllerRegistration.toRecord.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
2023-11-16 16:19:49 -08:00
Mickael Maison 832627fc78
MINOR: Various cleanups in metadata (#14734)
- Remove unused code, suppression
- Simplify/fix test assertions
- Javadoc cleanups

Reviewers: Josep Prat <josep.prat@aiven.io>
2023-11-14 09:25:09 +01:00
Proven Provenzano fa472d26a5 MINOR: Update BrokerRegistration to use a Builder
Update BrokerRegistration to use a Builder. This fixes the proliferation of different constructors,
and makes it clear what arguments are being used where.

Reviewers: Colin P. McCabe <cmccabe@confluent.io>
2023-11-09 10:08:31 -08:00
Colin Patrick McCabe 7060c08d6f
MINOR: Rewrite the meta.properties handling code in Java and fix some issues #14628 (#14628)
meta.properties files are used by Kafka to identify log directories within the filesystem.
Previously, the code for handling them was in BrokerMetadataCheckpoint.scala. This PR rewrites the
code for handling them as Java and moves it to the apache.kafka.metadata.properties namespace. It
also gets rid of the separate types for v0 and v1 meta.properties objects. Having separate types
wasn't so bad back when we had a strict rule that zk clusters used v0 and kraft clusters used v1.
But ZK migration has blurred the lines. Now, a zk cluster may have either v0 or v1, if it is
migrating, and a kraft cluster may have either v0 or v1, at any time.

The new code distinguishes between an individual meta.properties file, which is represented by
MetaProperties, and a collection of meta.properties files, which is represented by
MetaPropertiesEnsemble. It is useful to have this distinction, because in JBOD mode, even if some
log directories are inaccessible, we can still use the ensemble to extract needed information like
the cluster ID. (Of course, even when not in JBOD mode, KRaft servers have always been able to
configure a metadata log directory separate from the main log directory.)

Since we recently added a unique directory.id to each meta.properties file, the previous convention
of passing a "canonical" MetaProperties object for the cluster around to various places in the code
needs to be revisited. After all, we can no longer assume all of the meta.properties files are the
same. This PR fixes these parts of the code. For example, it fixes the constructors of
ControllerApis and RaftManager to just take a cluster ID, rather than a MetaProperties object. It
fixes some other parts of the code, like the constructor of SharedServer, to take a
MetaPropertiesEnsemble object.

Another goal of this PR was to centralize meta.properties validation a bit more and make it
unit-testable. For this purpose, the PR adds MetaPropertiesEnsemble.verify, and a few other
verification methods. These enforce invariants like "the metadata directory must be readable," and
so on.

Reviewers: Igor Soarez <soarez@apple.com>, David Arthur <mumrah@gmail.com>, Divij Vaidya <diviv@amazon.com>, Proven Provenzano <pprovenzano@confluent.io>
2023-11-09 09:32:35 -08:00
Calvin Liu 505e5b3eaa
KAFKA-15584: Leader election with ELR (#14593)
The patch includes the following changes as part of KIP-966

* Allow ISR shrink to empty
* Allow leader election with ELR members
* Allow electing the last known leader

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
2023-11-06 17:21:51 -05:00
Apoorv Mittal a53147e7d9
KAFKA-15673: Adding client metrics resource types (KIP-714) (#14621)
This PR adds resources to store and handle client metrics needed for KIP-714.

Changes include:

Adding CLIENT_METRICS to resource type
Corresponding DYNAMIC client configurations in resources.
Changes to support dynamic loading of configuration on changes.
Changes to support API calls to fetch data stored against the new resource.
Test cases for the changes.

Reviewers: Andrew Schofield <andrew_schofield@uk.ibm.com>, Philip Nee <pnee@confluent.io>, Jun Rao <junrao@gmail.com>
2023-11-03 14:51:50 -07:00
Igor Soarez 0390d5b1a2
KAFKA-15355: Message schema changes (#14290)
Reviewers: Christo Lolov <lolovc@amazon.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
2023-11-02 09:46:05 -04:00
Paolo Patierno 2736a2e50a
KAFKA-15689: Logging skipped event when expected migration state is wrong (#14646)
As described in ticket KAFKA-15689, this PR fixes the logging of a migration event when the expected migration state is wrong.

Signed-off-by: Paolo Patierno <ppatierno@live.com>

Reviewers: Luke Chen <showuon@gmail.com>
2023-10-30 17:59:11 +08:00
Paolo Patierno 0c7d1fca92
Using INFO level for migration transition state logging (#14651)
Trivial PR to use the INFO level (instead of DEBUG) for logging the state transition during the ZooKeeper to KRaft migration.
I think it's a useful information to be logged without the need for the user to increase the logging level itself.

Signed-off-by: Paolo Patierno <ppatierno@live.com>

Reviewers: Luke Chen <showuon@gmail.com>, hudeqi <1217150961@qq.com>
2023-10-30 17:57:26 +08:00
David Arthur 37715862d7 KAFKA-15704: Set missing ZkMigrationReady field on ControllerRegistrationRequest
This field was missed by the initial KIP-919 PR(s). The result is that migrations can't begin since
the controllers will never become ready. This patch fixes that as well as pulls over some fixes
from the 3.6 branch.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-10-27 14:16:24 -07:00
David Arthur 339d2556c6
KAFKA-15605: Fix topic deletion handling during ZK migration (#14545)
This patch adds reconciliation logic to migrating ZK brokers to deal with pending topic deletions as well as missed StopReplicas.

During the hybrid mode of the ZK migration, the KRaft controller is asynchronously sending UMR and LISR to the ZK brokers to propagate metadata. Since this process is essentially "best effort" it is possible for a broker to miss a StopReplicas. The new logic lets the ZK broker examine its local logs compared with the full set of replicas in a "Full" LISR. Any local logs which are not present in the set of replicas in the request are removed from ReplicaManager and marked as "stray".

To avoid inadvertent data loss with this new behavior, the brokers do not delete the "stray" partitions. They will rename the directories and log warning messages during log recovery. It will be up to the operator to manually delete the stray partitions. We can possibly enhance this in the future to clean up old stray logs.

This patch makes use of the previously unused Type field on LeaderAndIsrRequest. This was added as part of KIP-516 but never implemented. Since its introduction, an implicit 0 was sent in all LISR. The KRaft controller will now send a value of 2 to indicate a full LISR (as specified by the KIP). The presence of this value acts as a trigger for the ZK broker to perform the log reconciliation.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-10-26 18:13:52 -04:00
Calvin Liu af747fbfed
KAFKA-15581: Introduce ELR (#14312)
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)

* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-10-19 14:05:15 -04:00
Calvin Liu 14029e2ddd
KAFKA-15582: Identify clean shutdown broker (#14465)
The PR includes:

* Added a new class of CleanShutdownFile which helps write and read from a clean shutdown file.
* Updated the BrokerRegistration API.
* Client side handling for the broker epoch.
* Minimum work on the controller side.

Reviewers: Jun Rao <junrao@gmail.com>
2023-10-19 10:25:23 -07:00
mannoopj da314ee48c
KAFKA-15532: non active controllers return 0 for ZkWriteBeforelag (#14478)
Since only the active controller is performing the dual-write to ZK during a migration, it should be the only controller
to report the ZkWriteBehindLag metric.

Currently, if the controller fails over during a migration, the previous active controller will incorrectly report its last
value for ZkWriteBehindLag forever. Instead, it should report zero.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2023-10-16 15:22:50 -07:00
Federico Valeri aec07f76d7
KAFKA-15537: Fix metadata downgrade documentation (#14484)
In KIP-778 we introduced the "unsafe" (lossy) downgrade in case metadata has changes in one of the versions between target and current, as defined in MetadataVersion.

The documentation says it is possible:

"Note that the cluster metadata version cannot be downgraded to a pre-production 3.0.x, 3.1.x, or 3.2.x version once it has been upgraded. However, it is possible to downgrade to production versions such as 3.3-IV0, 3.3-IV1, etc."

The command line tool shows that this doesn't work:

bin/kafka-features.sh --bootstrap-server :9092 downgrade --metadata 3.4 --unsafe
Could not downgrade metadata.version to 8. Invalid metadata.version 8. Unsafe metadata downgrade is not supported in this version.
1 out of 1 operation(s) failed.

In addition to unsafe, also safe metadata downgrades are not supported in practice. For example, when you upgrade to 3.5, you land on 3.5-IV2 as metadata version, which has metadata changes and won't let you to downgrade. This is true for every other release at the moment.

This change fixes the documentation to reflect that, and improves the error messages.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>, Jakub Scholz <github@scholzj.com>
2023-10-12 11:12:44 +08:00
Ritika Reddy bcfc9543d1
MINOR: Move TopicIdPartition class to server-common (#14418)
This patch moves the TopicIdPartition from the metadata module to the server-common module so it can be used by the group-coordinator module as well.

Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, David Jacot <djacot@confluent.io>
2023-09-28 13:55:44 -07:00
Ismael Juma 98febb989a
KAFKA-15485: Fix "this-escape" compiler warnings introduced by JDK 21 (1/N) (#14427)
This is one of the steps required for kafka to compile with Java 21.

For each case, one of the following fixes were applied:
1. Suppress warning if fixing would potentially result in an incompatible change (for public classes)
2. Add final to one or more methods so that the escape is not possible
3. Replace method calls with direct field access.

In addition, we also fix a couple of compiler warnings related to deprecated references in the `core` module.

See the following for more details regarding the new lint warning:
https://www.oracle.com/java/technologies/javase/21-relnote-issues.html#JDK-8015831

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Chris Egerton <chrise@aiven.io>
2023-09-24 05:59:29 -07:00
Alyssa Huang 2d262efb00
[MINOR] QuorumController tests use testToImage (#14405) 2023-09-22 14:50:20 -04:00
Colin Patrick McCabe 7d45d849f8
KAFKA-15458: Fully resolve endpoint information before registering controllers (#14376)
Endpoint information provided by KafkaConfig can be incomplete in two ways. One is that endpoints
using ephemeral ports will show up as using port 0. Another is that endpoints binding to 0.0.0.0
will show up with a null or blank hostname. Because we were not accounting for this in controller
registration, it was leading to a null pointer dereference when we tried to register a controller
using an endpoint defined as PLAINTEXT://:9092.

This PR adds a ListenerInfo class which can fix both of the causes of incomplete endpoint
information. It also handles serialization to and from various RPC and record formats.
This allows us to remove a lot of boilerplate code and standardize the handling of listeners
between BrokerServer and ControllerServer.

Reviewers: David Arthur <mumrah@gmail.com>
2023-09-20 11:44:00 -07:00
David Arthur b24ccd65b7
KAFKA-15441 Allow broker heartbeats to complete in metadata transaction (#14351)
This patch allows broker heartbeat events to be completed while a metadata transaction is in-flight.

More generally, this patch allows any RUNS_IN_PREMIGRATION event to complete while the controller
is in pre-migration mode even if the migration transaction is in-flight.

We had a problem with broker heartbeats timing out because they could not be completed while a large
ZK migration transaction was in-flight. This resulted in the controller fencing all the ZK brokers which 
has many undesirable downstream effects. 

Reviewers: Akhilesh Chaganti <akhileshchg@users.noreply.github.com>, Colin Patrick McCabe <cmccabe@apache.org>
2023-09-08 16:36:13 -04:00
Colin Patrick McCabe 41b695b6e3
KAFKA-15369: Implement KIP-919: Allow AC to Talk Directly with Controllers (#14306)
Implement KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add
Controller Registration. This KIP adds a new version of DescribeClusterRequest which is supported
by KRaft controllers. It also teaches AdminClient how to use this new DESCRIBE_CLUSTER request to
talk directly with the controller quorum. This is all gated behind a new MetadataVersion,
IBP_3_7_IV0.

In order to share the DESCRIBE_CLUSTER logic between broker and controller, this PR factors it out
into AuthHelper.computeDescribeClusterResponse.

The KIP adds three new errors codes: MISMATCHED_ENDPOINT_TYPE, UNSUPPORTED_ENDPOINT_TYPE, and
UNKNOWN_CONTROLLER_ID. The endpoint type errors can be returned from DescribeClusterRequest

On the controller side, the controllers now try to register themselves with the current active
controller, by sending a CONTROLLER_REGISTRATION request. This, in turn, is converted into a
RegisterControllerRecord by the active controller. ClusterImage, ClusterDelta, and all other
associated classes have been upgraded to propagate the new metadata. In the metadata shell, the
cluster directory now contains both broker and controller subdirectories.

QuorumFeatures previously had a reference to the ApiVersions structure used by the controller's
NetworkClient. Because this PR removes that reference, QuorumFeatures now contains only immutable
data. Specifically, it contains the current node ID, the locally supported features, and the list
of quorum node IDs in the cluster.

Reviewers: David Arthur <mumrah@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
2023-09-07 15:21:52 -07:00
David Arthur 65e2ecffab
KAFKA-15435 Fix counts in MigrationManifest (#14342)
Reviewers: Liu Zeyu <zeyu.luke@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-09-06 13:02:13 -04:00
Dimitar Dimitrov 78c59cd2b0
KAFKA-15052 Fix the flaky testBalancePartitionLeaders - part II (#13908)
A follow-up to https://github.com/apache/kafka/pull/13804.
This follow-up adds the alternative fix approach mentioned in
the PR above - bumping the session timeout used in the test
with 1 second.

Reproducing the flake-out locally has been much harder than
on the CI runs, as neither Gradle with Java 11 or Java 14 nor
IntelliJ with Java 14 could show it, but IntelliJ with Java 11
could occasionally reproduce the failure the first time
immediately after a rebuild. While I was unable to see the
failure with the bumped session timeout, the testing procedure
definitely didn't provide sufficient reassurance for the
fix as even without it often I'd see hundreds of consecutive
successful test runs when the first run didn't fail.

Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>
2023-09-04 17:02:32 +08:00
David Arthur f2d499e25a
KAFKA-15389: Don't publish until we have replayed at least one record (#14282)
When starting up a controller for the first time (i.e., with an empty log), it is possible for
MetadataLoader to publish an empty MetadataImage before the activation records of the controller
have been written. While this is not a bug, it could be confusing. This patch closes that gap by
waiting for at least one controller record to be committed before the MetadataLoader starts publishing
images.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-25 10:41:43 -07:00
Phuc-Hong-Tran 8d12c1175c
KAFKA-15152: Fix incorrect format specifiers when formatting string (#14026)
Reviewers: Divij Vaidya <diviv@amazon.com>

Co-authored-by: phuchong.tran <phuchong.tran@servicenow.com>
2023-08-24 19:38:45 +02:00
Ron Dagostino 8394ddc0d2
MINOR: Move delegation token support to Metadata Version 3.6-IV2 (#14270)
#14083 added support for delegation tokens in KRaft and attached that support to the existing
MetadataVersion 3.6-IV1. This patch moves that support into a separate MetadataVersion 3.6-IV2.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-22 16:04:53 -07:00
David Arthur 418b8a6e59
KAFKA-14538 Metadata transactions in MetadataLoader (#14208)
This PR contains three main changes:

- Support for transactions in MetadataLoader
- Abort in-progress transaction during controller failover
- Utilize transactions for ZK to KRaft migration

A new MetadataBatchLoader class is added to decouple the loading of record batches from the
publishing of metadata in MetadataLoader. Since a transaction can span across multiple batches (or
multiple transactions could exist within one batch), some buffering of metadata updates was needed
before publishing out to the MetadataPublishers. MetadataBatchLoader accumulates changes into a
MetadataDelta, and uses a callback to publish to the publishers when needed.

One small oddity with this approach is that since we can "splitting" batches in some cases, the
number of bytes returned in the LogDeltaManifest has new semantics. The number of bytes included in
a batch is now only included in the last metadata update that is published as a result of a batch.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-21 16:02:14 -07:00
Proven Provenzano c2759df067
KAFKA-15219: KRaft support for DelegationTokens (#14083)
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>
2023-08-19 14:01:08 -04:00
Colin Patrick McCabe adc16d0f31
KAFKA-14538: Implement KRaft metadata transactions in QuorumController
Implement the QuorumController side of KRaft metadata transactions.

As specified in KIP-868, this PR creates a new metadata version, IBP_3_6_IV1, which contains the
three new records: AbortTransactionRecord, BeginTransactionRecord, EndTransactionRecord.

In order to make offset management unit-testable, this PR moves it out of QuorumController.java and
into OffsetControlManager.java. The general approach here is to track the "last stable offset," which is
calculated by looking at the latest committed offset and the in-progress transaction (if any). When
a transaction is aborted, we revert back to this last stable offset. We also revert back to it when
the controller is transitioning from active to inactive.

In a follow-up PR, we will add support for the transaction records in MetadataLoader. We will also
add support for automatically aborting pending transactions after a controller failover.

Reviewers: David Arthur <mumrah@gmail.com>
2023-08-14 16:58:56 -07:00
Colin Patrick McCabe 9318b591d7
KAFKA-15318: Update the Authorizer via AclPublisher (#14169)
On the controller, move publishing acls to the Authorizer into a dedicated MetadataPublisher,
AclPublisher. This publisher listens for notifications from MetadataLoader, and receives only
committed data. This brings the controller side in line with how the broker has always worked. It
also avoids some ugly code related to publishing directly from the QuorumController. Most important
of all, it clears the way to implement metadata transactions without worrying about Authorizer
state (since it will be handled by the MetadataLoader, along with other metadata image state).

In AclsDelta, we can remove isSnapshotDelta. We always know when the MetadataLoader is giving us a
snapshot. Also bring AclsDelta in line with the other delta classes, where completeSnapshot
calculates the diff between the previous image and the next one. We don't use this delta (since we
just apply the image directly to the authorizer) but we should have it, for consistency.

Finally, change MockAclMutator to avoid the need to subclass AclControlManager.

Reviewers: David Arthur <mumrah@gmail.com>
2023-08-09 23:54:46 -07:00
David Arthur 32c39c8149
KAFKA-15263 Check KRaftMigrationDriver state in each event (#14115)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-07-28 13:02:47 -04:00
Colin Patrick McCabe 10bcd4fc7f
KAFKA-15213: provide the exact offset to QuorumController.replay (#13643)
Provide the exact record offset to QuorumController.replay() in all cases. There are several situations
where this is useful, such as logging, implementing metadata transactions, or handling broker
registration records.

In the case where the QC is inactive, and simply replaying records, it is easy to compute the exact
record offset from the batch base offset and the record index.

The active QC case is more difficult. Technically, when we submit records to the Raft layer, it can
choose a batch base offset later than the one we expect, if someone else is also adding records.
While the QC is the only entity submitting data records, control records may be added at any time.
In the current implementation, these are really only used for leadership elections. However, this
could change with the addition of quorum reconfiguration or similar features.

Therefore, this PR allows the QC to tell the Raft layer that a record append should fail if it
would have resulted in a batch base offset other than what was expected. This in turn will trigger a
controller failover. In the future, if automatically added control records become more common, we
may wish to have a more sophisticated system than this simple optimistic concurrency mechanism. But
for now, this will allow us to rely on the offset as correct.

In order that the active QC can learn what offset to start writing at, the PR also adds a new
RaftClient#endOffset function.

At the Raft level, this PR adds a new exception, UnexpectedBaseOffsetException. This gets thrown
when we request a base offset that doesn't match the one the Raft layer would have given us.
Although this exception should cause a failover, it should not be considered a fault. This
complicated the exception handling a bit and motivated splitting more of it out into the new
EventHandlerExceptionInfo class. This will also let us unit test things like slf4j log messages a
bit better.

Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>
2023-07-27 17:01:55 -07:00
David Arthur a900794ace
KAFKA-15196 Additional ZK migration metrics (#14028)
This patch adds several metrics defined in KIP-866:

* MigratingZkBrokerCount: the number of zk brokers registered with KRaft
* ZkWriteDeltaTimeMs: time spent writing MetadataDelta to ZK
* ZkWriteSnapshotTimeMs: time spent writing MetadataImage to ZK
* Adds value 4 for "ZK" to ZkMigrationState

Also fixes a typo in the metric name introduced in #14009 (ZKWriteBehindLag -> ZkWriteBehindLag)

Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-07-26 12:54:59 -04:00
David Arthur e794bc719a
MINOR: Add a Builder for KRaftMigrationDriver (#14062)
Reviewers: Justine Olshan <jolshan@confluent.io>
2023-07-25 16:05:04 -04:00
Colin Patrick McCabe c7de30f38b
KAFKA-15183: Add more controller, loader, snapshot emitter metrics (#14010)
Implement some of the metrics from KIP-938: Add more metrics for
measuring KRaft performance.

Add these metrics to QuorumControllerMetrics:
    kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount
    kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount
    kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount
    kafka.controller:type=KafkaController,name=NewActiveControllersCount

Create LoaderMetrics with these new metrics:
    kafka.server:type=MetadataLoader,name=CurrentMetadataVersion
    kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount

Create SnapshotEmitterMetrics with these new metrics:
    kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes
    kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs

Reviewers: Ron Dagostino <rndgstn@gmail.com>
2023-07-24 21:13:58 -07:00
Owen Leung 4981fa939d
KAFKA-14712: Produce correct error msg with correct metadataversion (#13773)
Fix the confusing error message in ImageWriterOptions

Reviewers: Luke Chen <showuon@gmail.com>, David Arthur <mumrah@gmail.com>
2023-07-24 10:37:23 +08:00
Hailey Ni 9e50f7cdd3
MINOR: Add ZK dual-write lag metric (#14009)
This patch adds ZKWriteBehindLag metric to the KafkaController mbean as specified in KIP-866

Reviewers: David Arthur <mumrah@gmail.com>
2023-07-16 21:23:01 -04:00