Commit Graph

519 Commits

Author SHA1 Message Date
Ken Huang 676e0f2ad6
KAFKA-19139 Plugin#wrapInstance should use LinkedHashMap instead of Map (#19519)
CI / build (push) Waiting to run Details
There will be an update to the PluginMetrics#metricName method: the type
of the tags parameter will be changed
from Map to LinkedHashMap.
This change is necessary because the order of metric tags is important
1. If the tag order is inconsistent, identical metrics may be treated as
distinct ones by the metrics backend
2. KAFKA-18390 is updating metric naming to use LinkedHashMap. For
consistency, we should follow the same approach here.

Reviewers: TengYao Chi <frankvicky@apache.org>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, lllilllilllilili
2025-04-30 10:43:01 +08:00
Colin Patrick McCabe 22b89b6413
KAFKA-19192; Old bootstrap checkpoint files cause problems updated servers (#19545)
Old bootstrap.metadata files cause problems with server that include
KAFKA-18601. When the server tries to read the bootstrap.checkpoint
file, it will fail if the metadata.version is older than 3.3-IV3
(feature level 7). This causes problems when these clusters are
upgraded.

This PR makes it possible to represent older MVs in BootstrapMetadata
objects without causing an exception. An exception is thrown only if we
attempt to access the BootstrapMetadata. This ensures that only the code
path in which we start with an empty metadata log checks that the
metadata version is 7 or newer.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma
 <ismael@juma.me.uk>, PoAn Yang <payang@apache.org>, Liu Zeyu
 <zeyu.luke@gmail.com>, Alyssa Huang <ahuang@confluent.io>
2025-04-24 15:43:35 -04:00
José Armando García Sancio b97a130c08
KAFKA-16538; Enable upgrading kraft version for existing clusters (#19416)
This change implements upgrading the kraft version from 0 to 1 in existing clusters.
Previously, clusters were formatted with either version 0 or version 1, and could not
be moved between them.

The kraft version for the cluster metadata partition is recorded using the
KRaftVersion control record. If there is no KRaftVersion control record
the default kraft version is 0.

The kraft version is upgraded using the UpdateFeatures RPC. These RPCs
are handled by the QuorumController and FeatureControlManager. This
change adds special handling in the FeatureControlManager so that
upgrades to the kraft.version are directed to
RaftClient#upgradeKRaftVersion.

To allow the FeatureControlManager to call
RaftClient#upgradeKRaftVersion is a non-blocking fashion, the kraft
version upgrade uses optimistic locking. The call to
RaftClient#upgradeKRaftVersion does validations of the version change.
If the validations succeeds, it generates the necessary control records
and adds them to the BatchAccumulator.

Before the kraft version can be upgraded to version 1, all of the
brokers and controllers in the cluster need to support kraft version 1.
The check that all brokers support kraft version 1 is done by the
FeatureControlManager. The check that all of the controllers support
kraft version is done by KafkaRaftClient and LeaderState.

When the kraft version is 0, the kraft leader starts by assuming that
all voters do not support kraft version 1. The leader discovers which
voters support kraft version 1 through the UpdateRaftVoter RPC. The
KRaft leader handles UpdateRaftVoter RPCs by storing the updated
information in-memory until the kraft version is upgraded to version 1.
This state is stored in LeaderState and contains the latest directory
id, endpoints and supported kraft version for each voter.

Only when the KRaft leader has received an UpdateRaftVoter RPC from all
of the voters will it allow the upgrade from kraft.version 0 to 1.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-04-22 16:02:51 -07:00
Colin Patrick McCabe c465abc458
KAFKA-19130: Do not add fenced brokers to BrokerRegistrationTracker on startup (#19454)
When the controller starts up (or becomes active after being inactive), we add all of the registered brokers to BrokerRegistrationTracker so that they will not be accidentally fenced the next time we are looking for a broker to fence. We do this because the state in BrokerRegistrationTracker is "soft state" (it doesn't appear in the metadata log), and the newly active controller starts off with no soft state. (Its soft state will be populated by the brokers sending heartbeat requests to it over time.)

In the case of fenced brokers, we are not worried about accidentally fencing the broker due to it being missing from
BrokerRegistrationTracker for a while (it's already fenced). Therefore, it should be reasonable to just not add fenced brokers to the tracker initially.

One case where this change will have a positive impact is for people running single-node demonstration clusters in combined KRaft mode. In that case, when the single-node cluster is taken down and restarted, it currently will have to wait about 9 seconds for the broker to come up and re-register. With this change, the broker should be able to re-register immediately (assuming the previous shutdown happened cleanly through controller shutdown.)

One possible negative impact is that if there is a controller failover, it will open a small window where a broker with the same ID as a fenced broker could re-register. However, our detection of duplicate broker IDs is best-effort (and duplicate broker IDs are an administrative mistake), so this downside seems acceptable.

Reviewers: Alyssa Huang <ahuang@confluent.io>, José Armando García Sancio <jsancio@apache.org>
2025-04-16 11:57:35 -07:00
Mickael Maison fb2ce76b49
KAFKA-18888: Add KIP-877 support to Authorizer (#19050)
This also adds metrics to StandardAuthorizer

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>
2025-04-15 19:40:24 +02:00
PoAn Yang 42771b6144
KAFKA-18845 Remove flaky tag on QuorumControllerTest#testUncleanShutdownBrokerElrEnabled (#19403)
It has been around two weeks since fixing
QuorumControllerTest#testUncleanShutdownBrokerElrEnabled PR
https://github.com/apache/kafka/pull/19240 was merged. There is no flaky
result after 2025/03/21, so it has enough evidence to prove the flaky is
fixed. It's good to remove flaky tag.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-15 23:53:08 +08:00
Xuan-Zhang Gong 17ab374bb5
KAFKA-15371 MetadataShell is stuck when bootstrapping (#19419)
issue link https://issues.apache.org/jira/browse/KAFKA-15371

## conclusion

This issue isn’t caused by differences between the `log` file and the
`checkpoint` file, but rather by the order in which asynchronous events
occur.

## reliably reproduce
In the current version, you can reliably reproduce this issue by adding
a small sleep in `SnapshotFileReader#handleNextBatch` , like this:
```
 private void handleNextBatch() {
        if (!batchIterator.hasNext()) {
            try {
                Thread.sleep(1000);
            } catch (InterruptedException e) {
                throw new RuntimeException(e);
            }
            beginShutdown("done");
            return;
        }
        FileChannelRecordBatch batch = batchIterator.next();
        if (batch.isControlBatch()) {
            handleControlBatch(batch);
        } else {
            handleMetadataBatch(batch);
        }
        scheduleHandleNextBatch();
        lastOffset = batch.lastOffset();
    }
```

you can download a test file [test checkpoint

file](https://github.com/user-attachments/files/19659636/00000000000000007169-0000000001.checkpoint.log)

⚠️: Please remove the .log extension after downloading, since GitHub
doesn’t allow uploading checkpoint files directly.

After change code  and gradle build ,  you can run
`bin/kafka-metadata-shell.sh --snapshot   ${your file path}`

You will only see a loading message in the console like this:  <img
width="248" alt="image"

src="https://github.com/user-attachments/assets/fe4b4eba-7a6a-4cee-9b56-c82a5fa02c89"
/>

## Cause of the Bug
After the `SnapshotFileReader startup`, it will enqueue the iterator’s
events to its own kafkaQueue.
The impontent method is: `SnapshotFileReader#scheduleHandleNextBatch`

When processing each batch of the iterator, it adds metadata events for
the batch to the kafkaQueue(different from the SnapshotFileReader.) of
the metadataLoader.  The impontent method is
`SnapshotFileReader#handleMetadataBatch` and
`MetadataLoader#handleCommit`

When the MetadataLoader processes a MetadataDelta, it checks whether the
high watermark has been updated. If not, it skips processing    The
impontent method is `MetadataLoader#maybePublishMetadata` and
`maybePublishMetadata#stillNeedToCatchUp`

The crucial high watermark update happens after the SnapshotFileReader’s
iterator finishes reading, using the cleanup task of its kafkaQueue.

So, if the MetadataLoader finishes processing all batches before the
high watermark is updated, the main thread will keep waiting.  <img
width="1088" alt="image"

src="https://github.com/user-attachments/assets/03daa288-ff39-49a3-bbc7-e7b5831a858b"
/>

<img width="867" alt="image"

src="https://github.com/user-attachments/assets/fc0770dd-de54-4f69-b669-ab4e696bd2a7"
/>

## Solution
If we’ve reached the last batch in the iteration, we update the high
watermark first before adding events to the MetadataLoader, ensuring
that MetadataLoader runs at least once after the watermark is updated.

After modifying the code, you’ll see the normal shell execution
behavior.

<img width="337" alt="image"

src="https://github.com/user-attachments/assets/2791d03c-81ae-4762-a015-4d6d9e526455"
/>

Reviewers: PoAn Yang <payang@apache.org>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-14 14:13:50 +08:00
Andrew Schofield 21a080f08c
KAFKA-16894: Define feature to enable share groups (#19293)
This PR proposes a switch to enable share groups for 4.1 (preview) and
4.2 (GA).

* `share.version=1` to indicate that share groups are enabled. This is
used as the switch for turning share groups on and off.

In 4.1, the default will be `share.version=0`. Then a user wanting to
evaluate the preview of KIP-932 would use `bin/kafka-features.sh
--bootstrap.server xxxx upgrade --feature share.version=1`.

In 4.2, the default will be `share.version=1`.

Reviewers: Jun Rao <junrao@gmail.com>
2025-04-11 12:14:38 +01:00
Alyssa Huang 6e446f0b05
KAFKA-19047: Allow quickly re-registering brokers that are in controlled shutdown (#19296)
Allow re-registration of brokers with active sessions if the previous broker registration was in controlled shutdown.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Reviewers: José Armando García Sancio <jsancio@apache.org>, David Mao <dmao@confluent.io>
2025-04-08 13:39:04 -07:00
PoAn Yang dcf6f9d4c9
MINOR: remove unused function BrokerRegistration#isMigratingZkBroker (#19330)
The `BrokerRegistration#isMigratingZkBroker` is not used by any
production function. Remove it.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-09 01:56:46 +08:00
Dmitry Werner 4144290335
MINOR: Cleanup metadata module (#18937)
Removed unused code and fixed IDEA warnings.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-31 17:46:21 +08:00
José Armando García Sancio 82de719fff
MINOR; Improve error message for the storage format command (#19210)
Minor improvement on the error output for the storage format command. Suggests changes for a valid storage format command.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-03-23 19:30:57 -04:00
David Arthur 8fa3856473
MINOR Mar 19 flaky tests (#19248)
CoordinatorRequestManagerTest#testMarkCoordinatorUnknownLoggingAccuracy
has become flaky again. Last 30 days report shows a sudden re-occurrence


https://develocity.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.tags=github,trunk,not:flaky,not:new&search.tasks=test&search.timeZoneId=America%2FNew_York&tests.container=org.apache.kafka.clients.consumer.internals.CoordinatorRequestManagerTest&tests.sortField=FLAKY#

Also mark QuorumControllerTest.testMinIsrUpdateWithElr as flaky.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-03-21 09:26:13 -04:00
Calvin Liu 1c582a4a35
KAFKA-18954: Add ELR election rate metric (#19180)
Add a metric to track the number of election is done using ELR.
https://issues.apache.org/jira/browse/KAFKA-18954

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Justine Olshan
<jolshan@confluent.io>
2025-03-20 15:37:49 -07:00
PoAn Yang 71875ec58e
KAFKA-18845: Fix flaky QuorumControllerTest#testUncleanShutdownBrokerElrEnabled (#19240)
There're two root causes:
1. When we unclean shutdown `brokerToBeTheLeader`, we didn't wait for
the result. That means when we send heartbeat to unfence broker, it has
chance to use stale broker epoch to send the request. [0]
2. We use different replica directory to unclean shutdown broker. Even
if broker is unfenced, it cannot get an online directory, so the
`brokerToBeTheLeader` cannot be elected as a new leader. [1]


[0]
a5325e029e/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java (L484-L497)

[1]
a5325e029e/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java (L2470-L2477)

Reviewers: David Arthur <mumrah@gmail.com>
2025-03-20 13:37:09 -04:00
PoAn Yang da46cf6e79
KAFKA-17565 Move MetadataCache interface to metadata module (#18801)
### Changes

* Move MetadataCache interface to metadata module and change Scala
function to Java.
* Remove functions `getTopicPartitions`, `getAliveBrokers`,
`topicNamesToIds`, `topicIdInfo`, and `getClusterMetadata` from
MetadataCache interface, because these functions are only used in test
code.

### Performance

* ReplicaFetcherThreadBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.fetcher.ReplicaFetcherThreadBenchmark
  ```
  * trunk
  ```
Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 2 4775.490 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 2 25730.790 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 2 55334.206 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 2 488427.547 ns/op
  ```
  * branch
  ```
Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 2 4825.219 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 2 25985.662 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 2 56056.005 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 2 497138.573 ns/op
  ```

* KRaftMetadataRequestBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.metadata.KRaftMetadataRequestBenchmark
  ```
  * trunk
  ```
Benchmark (partitionCount) (topicCount) Mode Cnt Score Error Units
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 500
avgt 2 884933.558 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 1000
avgt 2 1910054.621 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 5000
avgt 2 21778869.337 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 500
avgt 2 1537550.670 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 1000
avgt 2 3168237.805 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 5000
avgt 2 29699652.466 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 500
avgt 2 3501483.852 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 1000
avgt 2 7405481.182 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 5000
avgt 2 55839670.124 ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 500 avgt 2 333.667
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 1000 avgt 2 339.685
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 5000 avgt 2 334.293
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 500 avgt 2 329.899
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 1000 avgt 2 347.537
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 5000 avgt 2 332.781
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 500 avgt 2 327.085
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 1000 avgt 2 325.206
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 5000 avgt 2 316.758
ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 500 avgt 2 7.569 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 1000 avgt 2 7.565 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 5000 avgt 2 7.574 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 500 avgt 2 7.568 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 1000 avgt 2 7.557 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 5000 avgt 2 7.585 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 500 avgt 2 7.560 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 1000 avgt 2 7.554 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 5000 avgt 2 7.574 ns/op
  ```
  * branch
  ```
Benchmark (partitionCount) (topicCount) Mode Cnt Score Error Units
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 500
avgt 2 910337.770 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 1000
avgt 2 1902351.360 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 5000
avgt 2 22215893.338 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 500
avgt 2 1572683.875 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 1000
avgt 2 3188560.081 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 5000
avgt 2 29984751.632 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 500
avgt 2 3413567.549 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 1000
avgt 2 7303174.254 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 5000
avgt 2 54293721.640 ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 500 avgt 2 318.335
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 1000 avgt 2 331.386
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 5000 avgt 2 332.944
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 500 avgt 2 340.322
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 1000 avgt 2 330.294
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 5000 avgt 2 342.154
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 500 avgt 2 341.053
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 1000 avgt 2 335.458
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 5000 avgt 2 322.050
ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 500 avgt 2 7.538 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 1000 avgt 2 7.548 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 5000 avgt 2 7.545 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 500 avgt 2 7.597 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 1000 avgt 2 7.567 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 5000 avgt 2 7.558 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 500 avgt 2 7.559 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 1000 avgt 2 7.615 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 5000 avgt 2 7.562 ns/op
  ```

* PartitionMakeFollowerBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.partition.PartitionMakeFollowerBenchmark
  ```
  * trunk
  ```
Benchmark Mode Cnt Score Error Units
PartitionMakeFollowerBenchmark.testMakeFollower avgt 2 158.816 ns/op
  ```
  * branch
  ```
Benchmark Mode Cnt Score Error Units
PartitionMakeFollowerBenchmark.testMakeFollower avgt 2 160.533 ns/op
  ```

* UpdateFollowerFetchStateBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.partition.UpdateFollowerFetchStateBenchmark
  ```
  * trunk
  ```
Benchmark Mode Cnt Score Error Units
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBench avgt 2
4975.261 ns/op
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBenchNoChange
avgt 2 4880.880 ns/op
  ```
  * branch
  ```
Benchmark Mode Cnt Score Error Units
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBench avgt 2
5020.722 ns/op
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBenchNoChange
avgt 2 4878.855 ns/op
  ```


* CheckpointBench
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.server.CheckpointBench
  ```
  * trunk
  ```
Benchmark (numPartitions) (numTopics) Mode Cnt Score Error Units
CheckpointBench.measureCheckpointHighWatermarks 3 100 thrpt 2 0.997
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 1000 thrpt 2 0.703
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 2000 thrpt 2 0.486
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 100 thrpt 2 1.038
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 1000 thrpt 2 0.734
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 2000 thrpt 2 0.637
ops/ms
  ```
  * branch
  ```
Benchmark (numPartitions) (numTopics) Mode Cnt Score Error Units
CheckpointBench.measureCheckpointHighWatermarks 3 100 thrpt 2 0.990
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 1000 thrpt 2 0.659
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 2000 thrpt 2 0.508
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 100 thrpt 2 0.923
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 1000 thrpt 2 0.736
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 2000 thrpt 2 0.637
ops/ms
  ```

* PartitionCreationBench
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.server.PartitionCreationBench
  ```
  * trunk
  ```
Benchmark (numPartitions) (useTopicIds) Mode Cnt Score Error Units
PartitionCreationBench.makeFollower 20 false avgt 2 5.997 ms/op
PartitionCreationBench.makeFollower 20 true avgt 2 6.961 ms/op
  ```
  * branch
  ```
Benchmark (numPartitions) (useTopicIds) Mode Cnt Score Error Units
PartitionCreationBench.makeFollower 20 false avgt 2 6.212 ms/op
PartitionCreationBench.makeFollower 20 true avgt 2 7.005 ms/op
  ```

Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-17 23:59:11 +08:00
Sanskar Jhajharia 766caaa551
MINOR: Clean up metadata module (#19069)
Given that now we support Java 17 on our brokers, this PR replace the
use of the following in metadata module:

Collections.singletonList() and Collections.emptyList() with List.of()
Collections.singletonMap() and Collections.emptyMap() with Map.of()
Collections.singleton() and Collections.emptySet() with Set.of()

Reviewers: David Arthur <mumrah@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-16 03:09:40 +08:00
José Armando García Sancio d04efca493
KAFKA-18979; Report correct kraft.version in ApiVersions (#19205)
Skip kraft.version when applying FeatureLevelRecord records. The kraft.version is stored as control records and not as metadata records. This solution has the benefits of removing from snapshots any FeatureLevelRecord for kraft.version that was incorrectly written to the log and allows ApiVersions to report the correct finalized kraft.version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-03-13 18:39:24 -04:00
PoAn Yang d171ff08a7
KAFKA-18858 Refactor FeatureControlManager to avoid using uninitialized MV (#19040)
The `FeatureControlManager` used `MetadataVersion#LATEST_PRODUCTION` as uninitialized MV. This makes other component may get a stale MV. In production code, the `FeatureControlManager` set MV when replaying `FeatureLevelRecord`, so we can set `Optional.empty()` as uninitialized MV. If other components get an empty result, the `FeatureLevelRecord` throws an exception like `FeaturesImage`.

Unit test:
* FeatureControlManagerTest#testMetadataVersion: test getting MetadataVersion
* before and after replaying FeatureLevelRecord.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-13 23:37:41 +08:00
David Arthur 0ebc3e83c5
MINOR Mar 12 Flaky tests (#19190)
Mark the following tests as flaky:

* StickyAssignorTest > testLargeAssignmentAndGroupWithUniformSubscription
* DeleteSegmentsByRetentionTimeTest
* QuorumControllerTest > testUncleanShutdownBrokerElrEnabled

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-12 13:47:35 -04:00
Lucas Brutschy fc2e3dfce9
MINOR: Disallow unused local variables (#18963)
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>
2025-03-10 09:37:35 +01:00
PoAn Yang a5e5e2dcd5
KAFKA-18706 Move AclPublisher to metadata module (#18802)
Move AclPublisher to org.apache.kafka.metadata.publisher package.

Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 21:00:33 +08:00
Colin Patrick McCabe 343bc995f4
KAFKA-18920: The kcontrollers must set kraft.version in ApiVersionsResponse (#19127)
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>
2025-03-07 13:46:46 -08:00
Calvin Liu db38bef076
KAFKA-18940: fix electionWasClean (#19156)
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>
2025-03-07 11:04:06 -08:00
Kevin Wu d04cddeb71
MINOR: Allow for configurable delay for periodic tasks (#19143)
This patch allows for the immediatePeriodNs to be passed in when creating a periodic task

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-03-07 11:24:34 -05:00
co63oc 3d7ac0c3d1
MINOR: Fix typos in multiple files (#19102)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-05 14:27:32 +00:00
Kuan-Po Tseng cbd72cc216
KAFKA-14121: AlterPartitionReassignments API should allow callers to specify the option of preserving the replication factor (#18983)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi <kitingiao@gmail.com>
2025-03-05 11:23:12 +00:00
ClarkChen 42a200bd39
KAFKA-18907 Add suitable error message when the appended value is too larger (#19070)
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>
2025-03-03 03:23:01 +08:00
Xuan-Zhang Gong 45f932819e
KAFKA-18864:remove the Evolving tag from stable public interfaces (#19036)
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>
2025-02-28 13:24:24 -08:00
Calvin Liu f6ec7df27e
MINOR: correct an ELR test name in ActivationRecordsGeneratorTest (#19044)
`testActivationMessageForEmptyLogAtMv3_6WithTransactionAndElr` is tested
with 4.0IV1, so the name should be corrected.

Reviewers: Jun Rao <junrao@apache.org>
2025-02-27 14:47:56 -08:00
PoAn Yang 88a23dab3e
KAFKA-18844: Stale features information in QuorumController#registerBroker (#18997)
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>
2025-02-27 09:30:51 -08:00
Ismael Juma 38c984307c
MINOR: Test showing MetadataLoader waits until metadata version is known (#19012)
Reviewers: David Arthur <mumrah@gmail.com>
2025-02-24 08:38:45 -08:00
Calvin Liu 8f13e7c207
MINOR: Move the ELR default version to 4.1 (#18954)
- 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>
2025-02-21 16:13:11 +01:00
Matthias J. Sax 538a60e1b3
MINOR: disallow rawtypes and fail build (#18877)
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>
2025-02-19 13:11:49 -08:00
Calvin Liu f85c7d4696
MINOR: Fix incorrect return value from upgradeFeatures #18958
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-02-19 09:41:06 -08:00
Ismael Juma 3dba3125e9
KAFKA-18601: Assume a baseline of 3.3 for server protocol versions (#18845)
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>
2025-02-19 05:35:42 -08:00
Jhen-Yung Hsu d0e516a872
KAFKA-18803 The acls would appear at the wrong level of the metadata shell "tree" (#18916)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 03:53:18 +08:00
David Jacot aec0e555be
MINOR: Mark IBP_4_0_IV3 as production ready! (#18902)
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>
2025-02-14 08:17:11 -08:00
Calvin Liu e7a2af8414
KAFKA-18634: Fix ELR metadata version issues (#18680)
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>
2025-02-13 23:40:31 -08:00
S.Y. Wang c2b38503f2
MINOR: Fix typo in ClusterControlManager (#18886)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-13 19:06:30 +08:00
Ken Huang 70adf746c4
KAFKA-18225 ClientQuotaCallback#updateClusterMetadata is unsupported by kraft (#18196)
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>
2025-02-11 01:03:02 +08:00
PoAn Yang 21645ebf0b
KAFKA-18705: Move ConfigRepository to metadata module (#18784)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Christo Lolov <lolovc@amazon.com>
2025-02-05 10:13:36 +00:00
Calvin Liu ad031b99d3
KAFKA-18635: reenable the unclean shutdown detection (#18277)
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>
2025-02-03 22:26:57 -08:00
kevin-wu24 184b891871
KAFKA-16524; Metrics for KIP-853 (#18304)
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>
2025-01-30 18:35:01 -05:00
Calvin Liu fdbed6c458
KAFKA-18649: complete ClearElrRecord handling (#18708)
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>
2025-01-29 15:07:44 -08:00
Calvin Liu 6235a73622
KAFKA-16540: Clear ELRs when min.insync.replicas is changed. (#18148)
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>
2025-01-24 10:57:33 -08:00
TengYao Chi fa2df3bca7
KAFKA-18559 Cleanup FinalizedFeatures (#18593)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-24 19:39:01 +08:00
Ken Huang 3d49159c84
KAFKA-18598: Remove ControllerMetadataMetrics ZK-related Metrics (#18629)
Reviewers: Christo Lolov <lolovc@amazon.com>
2025-01-21 15:23:39 +00:00
Ismael Juma 87b37a4065
KAFKA-14552: Assume a baseline of 3.0 for server protocol versions (#18497)
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>
2025-01-20 13:51:44 -08:00
Ismael Juma d4aee71e36
KAFKA-18465: Remove MetadataVersions older than 3.0-IV1 (#18468)
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>
2025-01-11 09:42:39 -08:00
Calvin Liu ec49a60e4f
KAFKA-16540: enforce min.insync.replicas config invariants for ELR (#17952)
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>
2025-01-08 13:42:25 -08:00
mingdaoy c40cc5740f
KAFKA-18408 tweak the 'tag' field for BrokerHeartbeatRequest.json, BrokerRegistrationChangeRecord.json and RegisterBrokerRecord.json (#18421)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 04:16:59 +08:00
David Arthur c4840f5e93
KAFKA-16446: Improve controller event duration logging (#15622)
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>
2025-01-06 13:34:46 -08:00
Ismael Juma 409a43eff7
MINOR: Collection/Option usage simplification via methods introduced in Java 9 & 11 (#18305)
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>
2025-01-03 16:13:39 -08:00
Ismael Juma d6f24d3665
Use `instanceof` pattern to avoid explicit cast (#18373)
This feature was introduced in Java 16.

Reviewers: David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-01-02 09:32:51 -08:00
Justine Olshan 8bd3746e0c
KAFKA-17705: Add Transactions V2 system tests and mark as production ready (#18132)
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>
2024-12-21 14:01:54 -08:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

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

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

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
Gantigmaa Selenge 747dc172e8
KIP-1073: Return fenced brokers in DescribeCluster response (#17524)
mplementation of KIP-1073: Return fenced brokers in DescribeCluster response.
Add new unit and integration tests for describeCluster.

Reviewers: Luke Chen <showuon@gmail.com>
2024-12-13 10:58:11 +08:00
Nick Guo 671cbedc1b
KAFKA-18219 Use INFO level instead of ERROR after successfully performing an unclean leader election (#18159)
Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-13 03:57:14 +08:00
TengYao Chi 772aa241b2
KAFKA-18136: Remove zk migration from code base (#18016)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-12 18:34:29 +01:00
David Mao 0ff55c316a
KAFKA-18106: Generate LeaderAndIsrUpdates on unclean shutdown (#18045)
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>
2024-12-05 16:19:05 -08:00
Dongnuo Lyu e30edb3eff
KAFKA-18052: Decouple the dependency of feature stable version to the metadata version (#17886)
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).

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

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

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2024-12-05 11:07:47 -08:00
Ken Huang 2b43c49f51
KAFKA-18050 Upgrade the checkstyle version to 10.20.2 (#17999)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-05 10:59:18 +08:00
Colin Patrick McCabe a8cdbaf4b3
KAFKA-18138: The controller must add all extant brokers to BrokerHeartbeatTracker when activating (#18009)
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>
2024-12-03 10:33:52 -05:00
Calvin Liu 2b2b3cd355
KAFKA-18062: use feature version to enable ELR (#17867)
Replace the ELR static config with feature version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-26 14:40:23 -08:00
PoAn Yang 98d47f47ef
KAFKA-18028 the effective kraft version of --no-initial-controllers should be 1 rather than 0 (#17836)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-27 01:45:11 +08:00
Colin Patrick McCabe cd36d64535
KAFKA-18051: Disallow creating ACLs with principals that do not contain a colon (#17883)
Kafka Principals must contain a colon. We should enforce this in createAcls.

Reviewers: David Arthur <mumrah@gmail.com>
2024-11-22 16:50:33 -08:00
Colin Patrick McCabe 130bf1054b
MINOR: some minor cleanups in the quorum controller. (#17819)
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>
2024-11-18 11:15:38 -08:00
Colin Patrick McCabe 085b27ec6e
KAFKA-17987 Remove assorted ZK-related files (#17768)
Remove zookeeper files in bin:
- bin/zookeeper-security-migration.sh
- bin/zookeeper-server-start.sh
- bin/zookeeper-server-stop.sh
- bin/zookeeper-shell.sh

Remove files used to configure Kafka in zookeeper mode in config:
- config/server.properties
- config/zookeeper.properties

Remove ZK references from all remaining Kafka configuration files.

Remove ZK references from all log4j.properties files.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-13 20:32:18 +08:00
kevin-wu24 ebb3202e01
KAFKA-16964 Integration tests for adding and removing voters (#17582)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-05 03:09:37 +08:00
Mahsa Seifikar b864a66439
MINOR: Add logging for ReplicationControlManager topic deletion (#17617)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-01 12:24:22 -07:00
Jonah Hooper 18b8b992f9
[KAFKA-17870] Fail CreateTopicsRequest if total number of partitions exceeds 10k (#17604)
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>
2024-10-31 13:54:03 -07:00
Mickael Maison d7135b2a5b
MINOR: Various cleanups in metadata (#17633)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-31 02:48:33 +08:00
Colin Patrick McCabe 14a9130f6f
KAFKA-17793: Improve kcontroller robustness against long delays (#17502)
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>
2024-10-28 08:36:07 -07:00
Kuan-Po Tseng edb623cf67
MINOR: Remove unused method in BrokerRegistration (#17568)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-23 02:12:20 +08:00
Colin Patrick McCabe e3751a838c
KAFKA-17794: Add some formatting safeguards for KIP-853 (#17504)
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>
2024-10-21 10:06:41 -07:00
Eric Chang 6b28e81ba1
KAKFA-17173 move quota config params from KafkaConfig to QuotaConfig (#17505)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-19 18:01:06 +08:00
Gaurav Narula b03fe66cfe
KAFKA-17759 Remove Utils.mkSet (#17460)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 21:20:43 +08:00
Chia-Chuan Yu b2380d7bf6
KAFKA-17772 Remove inControlledShutdownBrokers(Set<Integer>) and unfenceBrokers(Set<Integer>) from ReplicationControlManagerTest (#17466)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 17:14:12 +08:00
kevin-wu24 167e2f71f0
KAFKA-17713: Don't generate snapshot when published metadata is not batch aligned (#17398)
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>
2024-10-10 13:23:14 -04:00
TengYao Chi 924c1081dc
KAFKA-17415 Avoid overflow of expired timestamp (#17026)
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>
2024-10-07 01:43:43 +08:00
Colin Patrick McCabe 85bfdf4127
KAFKA-17613: Remove ZK migration code (#17293)
Remove the controller machinery for doing ZK migration in Kafka 4.0.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-10-03 12:01:14 -07:00
Justine Olshan 49d7ea6c6a
KAFKA-16308 [3/N]: Introduce feature dependency validation to UpdateFeatures command (#16443)
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>
2024-10-01 14:21:38 -07:00
Chung, Ming-Yen e136d7611c
KAFKA-17656 Replace string concatenation with parameterized logging for PartitionChangeBuilder (#17334)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-02 01:53:39 +08:00
Alyssa Huang 68b9770506
KAFKA-17608, KAFKA-17604, KAFKA-16963; KRaft controller crashes when active controller is removed (#17146)
This change fixes a few issues.

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

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

Reviewers: José Armando García Sancio <jsancio@apache.org>, Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@apache.org>
2024-09-26 13:56:19 -04:00
Colin Patrick McCabe 7c429f3514
KAFKA-17612 Remove some tests that only apply to ZK mode or migration (#17276)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-26 20:41:29 +08:00
Colin Patrick McCabe d3936365bf
KAFKA-16468: verify that migrating brokers provide their inter.broker.listener (#17159)
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>
2024-09-13 09:18:24 -07:00
David Arthur 0e30209f01
KAFKA-17506 KRaftMigrationDriver initialization race (#17147)
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>
2024-09-11 10:41:49 -04:00
David Arthur 1fd1646eb9
KAFKA-15648 Update leader volatile before handleLeaderChange in LocalLogManager (#17118)
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>
2024-09-06 13:54:03 -04:00
David Jacot c977bfdd3c
KAFKA-17413; Re-introduce `group.version` feature flag (#17013)
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>
2024-08-29 01:22:54 -07:00
Colin Patrick McCabe ca0cc355f6
KAFKA-12670: Support configuring unclean leader election in KRaft (#16866)
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>
2024-08-28 14:13:20 -07:00
TengYao Chi 4a485ddb71
KAFKA-17315 Fix the behavior of delegation tokens that expire immediately upon creation in KRaft mode (#16858)
In kraft mode, expiring delegation token (`expiryTimePeriodMs` < 0) has following different behavior to zk mode.

1. `ExpiryTimestampMs` is set to "expiryTimePeriodMs" [0] rather than "now" [1]
2. it throws exception directly if the token is expired already [2]. By contrast, zk mode does not. [3]

[0] 49fc14f611/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java (L316)
[1] 49fc14f611/core/src/main/scala/kafka/server/DelegationTokenManagerZk.scala (L292)
[2] 49fc14f611/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java (L305)
[3] 49fc14f611/core/src/main/scala/kafka/server/DelegationTokenManagerZk.scala (L293)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-25 07:29:58 +08:00
Dmitry Werner 6cad2c0d67
KAFKA-17370 Move LeaderAndIsr to metadata module (#16943)
isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toL
2024-08-22 15:47:09 +08:00
Alyssa Huang 0bb2aee838
KAFKA-17305; Check broker registrations for missing features (#16848)
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>
2024-08-21 11:14:56 -07:00
TengYao Chi 81f0b13a70
KAFKA-17238 Move VoterSet and ReplicaKey from raft.internals to raft (#16775)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-16 00:24:51 +08:00
José Armando García Sancio 0f7cd4dcde
KAFKA-17304; Make RaftClient API for writing to log explicit (#16862)
RaftClient API is changed to separate the batch accumulation (RaftClient#prepareAppend) from scheduling the append of accumulated batches (RaftClient#schedulePrepatedAppend) to the KRaft log. This change is needed to better match the controller's flow of replaying the generated records before replicating them. When the controller replay records it needs to know the offset associated with the record. To compute a table offset the KafkaClient needs to be aware of the records and their log position.

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

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

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

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2024-08-14 15:42:04 -04:00
DL1231 3a0efa2845
KAFKA-14510; Extend DescribeConfigs API to support group configs (#16859)
This patch extends the DescribeConfigs API to support group configs.

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
2024-08-14 06:37:57 -07:00
Colin Patrick McCabe 132e0970fb
KAFKA-17018: update MetadataVersion for the Kafka release 3.9 (#16841)
- 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>
2024-08-12 16:30:43 -07:00
Colin Patrick McCabe e1b2adea07
KAFKA-17190: AssignmentsManager gets stuck retrying on deleted topics (#16672)
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>
2024-08-10 12:31:45 +01:00
Josep Prat 4e862c0903
KAFKA-15875: Stops leak Snapshot in public methods (#16807)
* 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>
2024-08-08 20:05:47 +02:00