Mark the following tests as flaky:
* StickyAssignorTest > testLargeAssignmentAndGroupWithUniformSubscription
* DeleteSegmentsByRetentionTimeTest
* QuorumControllerTest > testUncleanShutdownBrokerElrEnabled
Reviewers: Andrew Schofield <aschofield@confluent.io>
Recently, we found a regression that could have been detected by static
analysis, since a local variable wasn't being passed to a method during
a refactoring, and was left unused. It was fixed in
[7a749b5](7a749b589f),
but almost slipped into 4.0. Unused variables are typically detected by
IDEs, but this is insufficient to prevent these kinds of bugs. This
change enables unused local variable detection in checkstyle for Kafka.
A few notes on the usage:
- There are two situations in which people actually want to have a local
variable but not use it. First, there are `for (Type ignored:
collection)` loops which have to loop `collection.length` number of
times, but that do not use `ignored` in the loop body. These are
typically still easier to read than a classical `for` loop. Second, some
IDEs detect it if a return value of a function such as `File.delete` is
not being used. In this case, people sometimes store the result in an
unused local variable to make ignoring the return value explicit and to
avoid the squiggly lines.
- In Java 22, unsued local variables can be omitted by using a single
underscore `_`. This is supported by checkstyle. In pre-22 versions,
IntelliJ allows such variables to be named `ignored` to suppress the
unused local variable warning. This pattern is often (but not
consistently) used in the Kafka codebase. This is, however, not
supported by checkstyle.
Since we cannot switch to Java 22, yet, and we want to use automated
detection using checkstyle, we have to resort to prefixing the unused
local variables with `@SuppressWarnings("UnusedLocalVariable")`. We have
to apply this in 11 cases across the Kafka codebase. While not being
pretty, I'd argue it's worth it to prevent bugs like the one fixed in
[7a749b5](7a749b589f).
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur
<mumrah@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bruno
Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>
The kafka controllers need to set kraft.version in their
ApiVersionsResponse messages according to the current kraft.version
reported by the Raft layer. Instead, currently they always set it to 0.
Also remove FeatureControlManager.latestFinalizedFeatures. It is not
needed and it does a lot of copying.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The electionWasClean should also consider if the election is done
through ELR. Otherwise, the metric uncleanLeaderElection will wrongly
count the ELR election
https://issues.apache.org/jira/browse/KAFKA-18940
Reviewers: Jun Rao <junrao@gmail.com>
In ZooKeeper mode, users can append configurations to create values larger than Short.MAX_VALUE. However, this behavior is disallowed in KRaft mode. Additionally, a server error is returned to users. Creating a value this large is rare, so we don't plan to fix it for KRaft. This PR aims to tweak the error message.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The purpose of this PR is to remove the `@InterfaceStability.Evolving` from classes that were created over a year ago.
Reviewers: Jun Rao <junrao@gmail.com>
`testActivationMessageForEmptyLogAtMv3_6WithTransactionAndElr` is tested
with 4.0IV1, so the name should be corrected.
Reviewers: Jun Rao <junrao@apache.org>
In https://github.com/apache/kafka/pull/16848, we added `kraft.version`
to finalized features and got finalized features outside controller
event handling thread. This may make finalized features stale when
processing `registerBroker` event. Also, some cases like
`QuorumControllerTest.testBalancePartitionLeaders` become flaky cause of
outdated MV. This PR moves finalized features back to controller event
handling thread to avoid the error.
Reviewers: Ismael Juma <ijuma@apache.org>, Jun Rao <junrao@gmail.com>,
Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
- ELR is enabled (ELRV_1) by default if the cluster is created with its bootstrap metadata version >= IBP_4_1_IV0.
- ELRV_1 can be manually enabled iff the metadata version is >= IBP_4_0_IV1.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>, David Jacot <djacot@confluent.io>
Cleanup code to avoid rawtype, and add suppressions where necessary.
Change the build to fail on rawtype warning.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
3.3.0 was the first KRaft release that was deemed production-ready and also
when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable
for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also
needs to be set to "3.3" or newer before upgrading).
Noteworthy changes:
1. `AlterPartition` no longer includes topic names, which makes it possible to
simplify `AlterParitionManager` logic.
2. Metadata versions older than `IBP_3_3_IV3` have been removed and
`IBP_3_3_IV3` is now the minimum version.
3. `MINIMUM_BOOTSTRAP_VERSION` has been removed.
4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`,
`isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and
`isInControlledShutdownStateSupported` - these are always `true` now.
Also removed related conditional code.
5. Removed default metadata version or metadata version fallbacks in
multiple places - we now fail-fast instead of potentially using an incorrect
metadata version.
6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord`
based on whether image is empty - this was a previously existing issue that
became more apparent after the changes in this PR.
7. Remove `ibp` parameter from `BootstrapDirectory`
8. A number of tests were not useful anymore and have been removed.
I will update the upgrade notes via a separate PR as there are a few things that
need changing and it would be easier to do so that way.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
This patch marks IBP_4_0_IV3 as production ready for the Apache Kafka 4.0 release. It also introduced IBP_4_1_IV0 as the next development version.
Reviewers: Justine Olshan <jolshan@confluent.io>
This patch cleans up the places that should not use MV to determine ELR is enabled marks 4.0IV1 stable.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
This commit ensures that the ClientQuotaCallback#updateClusterMetadata method is executed in KRaft mode. This method is triggered whenever a topic or cluster metadata change occurs. However, in KRaft mode, the current implementation of the updateClusterMetadata API is inefficient due to the requirement of creating a full Cluster object. To address this, a follow-up issue (KAFKA-18239) has been created to explore more efficient mechanisms for providing cluster information to the ClientQuotaCallback without incurring the overhead of a full Cluster object creation.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
We need to re-enable the unclean shutdown detection when in ELR mode, which was inadvertently removed during the development process.
Reviewers: David Mao <dmao@confluent.io>, Jun Rao <junrao@gmail.com>
This change implement some of the metrics enumerated in KIP-853.
The KafkaRaftMetrics object now exposes number-of-voters, number-of-observers and uncommitted-voter-change. The number-of-observers and uncommitted-voter-change metrics are only present on the active controller or leader, since it does not make sense for other replicas to report these metrics.
In order to make these two metrics thread-safe, KafkaRaftMetrics needs to be passed into LeaderState, and therefore QuorumState. This introduces a circularity since the KafkaRaftMetrics constructor takes in QuorumState. To break the circularity for now, the logic using QuorumState will be moved to the KafkaRaftMetrics#initialize method.
The BrokerServerMetrics object now exposes ignored-static-voters. The ControllerServerMetrics object now exposes IgnoredStaticVoters. To implement both metrics for "ignored static voters", this PR introduces the ExternalKRaftMetrics interface, which allows for higher layer metrics objects to be accessible within the raft module.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Implement ClearElrRecord handling in the TopicDelta. Also, the ReplicationControlManager should not merge updates if ELR/LastKnownElr are empty, becuase that will cause an unnecessary partition epoch bump.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
In order to maintain the integrity of replication, we need to clear the ELRs of affected partitions when min.insync.replicas is changed. This could happen at the topic level, or at a global level if the cluster level default is changed.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Kafka 4.0 will remove support for zk mode and will require conversion to kraft
before upgrading to 4.0. The minimum kraft version is 3.0 (aka 3.0-IV1).
This provides an opportunity to remove exclusively server side protocols versions
that only exist to allow direct upgrades from versions older than 3.0 or that are
used only by zk mode.
Since KRaft became production ready in 3.3, we should consider setting the
baseline to 3.3. But that requires more discussion and it can be done via a
separate change (KAFKA-18601).
Protocol changes:
* Remove RequestHeader v0 (only used by ControlledShutdown v0)
* Remove WriteTxnMarkers v0
* Remove all versions of ControlledShutdown, LeaderAndIsr, StopReplica, UpdateMetadata
In order to remove all versions safely, extend generator to support setting
"versions" to "none". In this case, we no longer generate the `*Data` classes,
but we still reserve the id for the relevant protocol api (so it doesn't get
accidentally used for something else). The protocol documentation is correct
after these changes.
We kept a simplified version of `LeaderAndIsr{Request|Response}` because
it's used by many tests that are still relevant in kraft mode. Once
KAFKA-18486 is done, it may be possible to remove it (I left a comment on
the ticket). Similarly, KAFKA-18487 may make it possible to remove
the introduced `StopReplicaPartitionState` (left a comment on that ticket too).
There are a number of places that were adjusted to include an
`ApiKeys.hasValidVersion` check.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.
Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
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>
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>
As discussed in #16657 (comment) , we should make logger as static to avoid creating multiple logger instances.
I use the regex private.*Logger.*LoggerFactory to search and check all the results if certain logs need to be static.
There are some exceptions that loggers don't need to be static:
1) The logger in the inner class. Since java8 doesn't support static field in the inner class.
https://github.com/apache/kafka/blob/trunk/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java#L3676
2) Custom loggers for each instance (non-static + non-final). In this case, multiple logger instances is actually really needed.
https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java#L166
3) The logger is initialized in constructor by LogContext. Many non-static but with final modifier loggers are in this category, that's why I use .*LoggerFactory to only check the loggers that are assigned initial value when declaration.
4) protected final Logger log = Logger.getLogger(getClass())
This is for subclass can do logging with subclass name instead of superclass name.
But in this case, if the log access modifier is private, the purpose cannot be achieved since subclass cannot access the log defined in superclass. So if access modifier is private, we can replace getClass() with <className>.class
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This change includes adding transaction.version (part of KIP-1022)
New transaction version 1 is introduced to support writing flexible fields in transaction state log messages.
Transaction version 2 is created in anticipation for further KIP-890 changes.
Neither are made production ready. Tests for the new transaction version and new MV are created.
Also include change to not report a feature as supported if the range is 0-0.
Reviewers: Jun Rao <junrao@apache.org>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
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>
When becoming the active KRaftMigrationDriver, there is another race condition similar to KAFKA-16171. This time, the race is due to a stale read from ZK. After writing to /controller and /controller_epoch, it is possible that a read on /migration is not linearized with the writes that were just made. In other words, we get a stale read on /migration. This leads to an inability to sync metadata to ZK due to incorrect zkVersion on the migration ZNode.
The non-linearizability of reads is in fact documented behavior for ZK, so we need to handle it.
To fix the stale read, this patch adds a write to /migration after updating /controller and /controller_epoch. This allows us to learn the correct zkVersion for the migration ZNode before leaving the BECOME_CONTROLLER state.
This patch also adds a check on the current leader epoch when running certain events in KRaftMigrationDriver. Historically, we did not include this check because it is not necessary for correctness. Writes to ZK are gated on the /controller_epoch zkVersion, and RPCs sent to brokers are gated on the controller epoch. However, during a time of rapid failover, there is a lot of processing happening on the controller (i.e., full metadata sync to ZK and full UMRs sent to brokers), so it is best to avoid running events we know will fail.
There is also a small fix in here to improve the logging of ZK operations. The log message are changed to past tense to reflect the fact that they have already happened by the time the log message is created.
Reviewers: Igor Soarez <soarez@apple.com>
Create 3 new metadata versions:
- 3.8-IV0, for the upcoming 3.8 release.
- 3.9-IV0, to add support for KIP-1005.
- 3.9-IV1, as the new release vehicle for KIP-966.
Create ListOffsetRequest v9, which will be used in 3.9-IV0 to support KIP-1005. v9 is currently an unstable API version.
Reviewers: Jun Rao <junrao@gmail.com>, Justine Olshan <jolshan@confluent.io>
Add field tieredEpoch to RemoteLogSegmentMetadata
Update relevant tests
Add two fields tieredEpoch and tieredState to TopicRecord.json
Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>
* 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>