If ELR is enabled, we need to set a cluster-level min.insync.replicas, and remove all broker-level overrides. The reason for this is that if brokers disagree about which partitions are under min ISR, it breaks the KIP-966 replication invariants. In order to enforce this, when the eligible.leader.replicas.version feature is turned on, we automatically remove all broker-level min.insync.replicas overrides, and create the required cluster-level override if needed. Similarly, if the cluster was created with eligible.leader.replicas.version enabled, we create a similar cluster-level record. In both cases, we don't allow setting overrides for individual brokers afterwards, or removing the cluster-level override.
Split ActivationRecordsGeneratorTest up into multiple test cases rather than having it be one giant test case.
Fix a bug in QuorumControllerTestEnv where we would replay records manually on objects, racing with the active controller thread. Instead, we should simply ensure that the initial bootstrap records contains what we want.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
There are times when the controller has a high event processing time, such as during startup, or when creating a topic with many partitions. We can see these processing times in the p99 metric (kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs), however it's difficult to see exactly which event is causing high processing time.
With DEBUG logs, we see every event along with its processing time. Even with this, it's a bit tedious to find the event with a high processing time.
This PR logs all events which take longer than 2 seconds at ERROR level. This will help identify events that are taking far too long, and which could be disruptive to the operation of the controller. The slow event logging looks like this:
```
[2024-12-20 15:03:39,754] ERROR [QuorumController id=1] Exceptionally slow controller event createTopics took 5240 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
Also, every 60 seconds, it logs some event time statistics, including average time, maximum time, and the name of the event which took the longest. This periodic message looks like this:
```
[2024-12-20 15:35:04,798] INFO [QuorumController id=1] In the last 60000 ms period, 333 events were completed, which took an average of 12.34 ms each. The slowest event was handleCommit[baseOffset=0], which took 41.90 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
An operator can disable these logs by adding the following to their log4j config:
```
org.apache.kafka.controller.EventPerformanceMonitor=OFF
```
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).
Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
Added transaction version 2 to some of the system tests. Also marking TV2 as production ready.
Also fixes the defaultVersion test.
Reviewers: Jun Rao <jun@confluent.io>
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>
mplementation of KIP-1073: Return fenced brokers in DescribeCluster response.
Add new unit and integration tests for describeCluster.
Reviewers: Luke Chen <showuon@gmail.com>
Generate LeaderAndISR change records when a broker re-registers and the quorum controller detects an unclean shutdown.
This is necessary to ensure that we perform the expected partition state transitions, eg: bumping leader epochs and so on.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
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>
The controller must add all extant brokers to BrokerHeartbeatTracker when activating. Otherwise, we
could end up in a situation where a broker fails exactly as a controller failover occurs, and we
never fence it.
Also, fix a bug where the slf4j logger object in PeriodicTaskControlManager was initialized as
though it belonged to OffsetControlManager.
Reviewers: David Mao <dmao@confluent.io>, David Arthur <mumrah@gmail.com>
BrokerHeartbeatManager.java: fix an outdated comment.
Move an inefficient test method that is O(num_brokers) from ClusterControlManager.java into ReplicationControlManagerTest.java, so that it doesn't accidentally get used in production code.
Remove QuorumController.ImbalanceSchedule, etc. since it is no longer used.
Move the initialization of OffsetControlManager later in the QuorumController constructor and add a comment explaining why it should come last. This doesn't fix any bugs currently, but it's a good practice for the future.
Reviewers: Mickael Maison <mickael.maison@gmail.com>
We fail the entire CreateTopicsRequest action if there are more than 10k total
partitions being created in this topic for this specific request. The usual pattern for
this API to try and succeed with some topics. Since the 10k limit applies to all topics
then no topic should be created if they all exceede it.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
As described in KIP-500, the Kafka controller monitors the liveness of each broker in the cluster. It gathers this information from heartbeats sent from the brokers themselves.
In some rare cases, the main controller thread may get blocked for several seconds at a time. In the current code, this will result in the controller being unable to update the last contact times for the brokers during this time.
This PR changes the controller heartbeat handling to be partially lockless. Specifically, the last contact time for each broker will be updated locklessly prior to the rest of the heartbeat handling. This will ensure that heartbeats always get through.
Additionally, this PR adds a PeriodicTaskControlManager to better manage periodic tasks. This should help handle the very common pattern where we want to schedule a background task at some frequency. We also want the background task to be immediately rescheduled if there is too much work to be done in one event.
Reviewers: Liu Zeyu <zeyu.luke@gmail.com>, David Arthur <mumrah@gmail.com>
KIP-853 adds support for dynamic KRaft quorums. This means that the quorum topology is
no longer statically determined by the controller.quorum.voters configuration. Instead, it
is contained in the storage directories of each controller and broker.
Users of dynamic quorums must format at least one controller storage directory with either
the --initial-controllers or --standalone flags. If they fail to do this, no quorum can be
established. This PR changes the storage tool to warn about the case where a KIP-853 flag has
not been supplied to format a KIP-853 controller. (Note that broker storage directories
can continue to be formatted without a KIP-853 flag.)
There are cases where we don't want to specify initial voters when formatting a controller. One
example is where we format a single controller with --standalone, and then dynamically add 4
more controllers with no initial topology. In this case, we want the 4 later controllers to grab
the quorum topology from the initial one. To support this case, this PR adds the
--no-initial-controllers flag.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Federico Valeri <fvaleri@redhat.com>
When MetadataBatchLoader handles a BeginTransactionRecord, it will publish the metadata that has seen so far and not publish again until the transaction is ended or aborted. This means a partial record batch can be published. If a snapshot is generated during this time, the currently published metadata may not align with the end of a record batch. This causes problems with Raft replication which expects a snapshot's offset to exactly precede a record batch boundary.
This patch enhances SnapshotGenerator to refuse to generate a snapshot if the metadata is not batch aligned.
Reviewers: David Arthur <mumrah@gmail.com>
Both ZK and KRaft modes do not handle overflow, so setting a large max lifetime results in a negative expired timestamp and negative max timestamp, which is unexpected behavior.
In this PR, we are only fixing the KRaft code since ZK will be removed soon.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This change includes:
1. Dependency checking when updating the feature (all request versions)
2. Returning top level error and no feature level errors if any feature failed to update and using this error for all the features in the response. (all request versions)
3. Returning only top level none error for v2 and beyond
Reviewers: Jun Rao <jun@confluent.io>
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>
When brokers undergoing ZK migration register with the controller, it should verify that they have
provided a way to contact them via their inter.broker.listener. Otherwise the migration will fail
later on with a more confusing error message.
Reviewers: David Arthur <mumrah@gmail.com>
There is a race condition between KRaftMigrationDriver running its first poll() and being notified by Raft about a leader change. If onControllerChange is called before RecoverMigrationStateFromZKEvent is run, we will end up getting stuck in the INACTIVE state.
This patch fixes the race by enqueuing a RecoverMigrationStateFromZKEvent from onControllerChange if the driver has not yet initialized. If another RecoverMigrationStateFromZKEvent was already enqueued, the second one to run will just be ignored.
Reviewers: Luke Chen <showuon@gmail.com>
Update the leader before calling handleLeaderChange and use the given epoch in LocalLogManager#prepareAppend. This should hopefully fix several flaky QuorumControllerTest tests.
Reviewers: José Armando García Sancio <jsancio@apache.org>
This patch re-introduces the `group.version` feature flag and gates the new consumer rebalance protocol with it. The `group.version` feature flag is attached to the metadata version `4.0-IV0` and it is marked as production ready. This allows system tests to pick it up directly by default without requiring to set `unstable.feature.versions.enable` in all of them. This is fine because we don't plan to do any incompatible changes before 4.0.
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Previously in KRaft mode, we could request an unclean leader election for a specific topic using
the electLeaders API. This PR adds an additional way to trigger unclean leader election when in
KRaft mode via the static controller configuration and various dynamic configurations.
In order to support all possible configuration methods, we have to do a multi-step configuration
lookup process:
1. check the dynamic topic configuration for the topic.
2. check the dynamic node configuration.
3. check the dynamic cluster configuration.
4. check the controller's static configuration.
Fortunately, we already have the logic to do this multi-step lookup in KafkaConfigSchema.java.
This PR reuses that logic. It also makes setting a configuration schema in
ConfigurationControlManager mandatory. Previously, it was optional for unit tests.
Of course, the dynamic configuration can change over time, or the active controller can change
to a different one with a different configuration. These changes can make unclean leader
elections possible for partitions that they were not previously possible for. In order to address
this, I added a periodic background task which scans leaderless partitions to check if they are
eligible for an unclean leader election.
Finally, this PR adds the UncleanLeaderElectionsPerSec metric.
Co-authored-by: Luke Chen showuon@gmail.com
Reviewers: Igor Soarez <soarez@apple.com>, Luke Chen <showuon@gmail.com>
When a broker tries to register with the controller quorum, its registration should be rejected if it doesn't support a feature that is currently enabled. (A feature is enabled if it is set to a non-zero feature level.) This is important for the newly added kraft.version feature flag.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, José Armando García Sancio <jsancio@apache.org>
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>
This patch extends the DescribeConfigs API to support group configs.
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
- Mark 3.9-IV0 as stable. Metadata version 3.9-IV0 should return Fetch version 17.
- Move ELR to 4.0-IV0. Remove 3.9-IV1 since it's no longer needed.
- Create a new 4.0-IV1 MV for KIP-848.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
In MetadataVersion 3.7-IV2 and above, the broker's AssignmentsManager sends an RPC to the
controller informing it about which directory we have chosen to place each new replica on.
Unfortunately, the code does not check to see if the topic still exists in the MetadataImage before
sending the RPC. It will also retry infinitely. Therefore, after a topic is created and deleted in
rapid succession, we can get stuck including the now-defunct replica in our subsequent
AssignReplicasToDirsRequests forever.
In order to prevent this problem, the AssignmentsManager should check if a topic still exists (and
is still present on the broker in question) before sending the RPC. In order to prevent log spam,
we should not log any error messages until several minutes have gone past without success.
Finally, rather than creating a new EventQueue event for each assignment request, we should simply
modify a shared data structure and schedule a deferred event to send the accumulated RPCs. This
will improve efficiency.
Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rndgstn@gmail.com>
* KAFKA-15875: Stops leak Snapshot in public methods
The Snapshot class is package protected but it's returned in
several public methods in SnapshotRegistry.
To prevent this accidental leakage, these methods are made
package protected as well. For getOrCreateSnapshot a new
method called IdempotentCreateSnapshot is created that returns void.
* Make builer package protected, replace <br> with <p>
Reviewers: Greg Harris <greg.harris@aiven.io>