Commit Graph

15343 Commits

Author SHA1 Message Date
Ismael Juma 811e406074 MINOR: Run CI with Java 24 and update documentation (instead of 23) 2025-03-18 19:50:02 -07:00
Kevin Wu a5325e029e
KAFKA-17431: Support invalid static configs for KRaft so long as dynamic configs are valid (#18949)
During broker startup, attempt to read dynamic configurations from latest local snapshot on disk. This will avoid most situations where the static configuration is not sufficient to start up, but the dynamic configuration would have been. The PR includes an integration test.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-03-18 14:23:23 -07:00
David Jacot 3ed590288f
MINOR: Add 4.0.0 to deps (#19229)
This patch adds 4.0 release to deps, dockerfile and vagrant.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-03-18 08:56:53 -07:00
David Arthur bb24d8596e
MINOR Fixup develocity report (#19179)
After our recent build task changes, this report no longer works. This
patch fixes the script to use the new test task and tags.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 10:17:44 -04:00
David Jacot 806aa55b3b
MINOR: Update docker_scan.yml with 4.0.0 (#19225)
Update docker_scan.yml with 4.0.0.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 11:09:36 +01:00
Ken Huang b805877705
KAFKA-18969 Rewrite ShareConsumerTest#setup and move to clients-integration-tests module (#19202)
Move share consumer to clients-integration-tests module and use `@BeforeEach` to setup

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 14:47:38 +08:00
Alyssa Huang 0ce87e7236
KAFKA-18955: Fix infinite loop and standardize options in MetadataSchemaCheckerTool (#19165)
- fix infinite loop in FieldSpecPairIterator.java 
- fix bug in Unifier.java that was resulting in verify-evolution and
verify-evolution-git to break in certain scenarios and probably generate
false positives
- verify-evolution-git command takes an absolute path like the other
commands
- verify-evolution arguments are more clear (path and parent_path)

Reviewers: Reviewers: Josep Prat <josep.prat@aiven.io>, mannoopj <139923522+mannoopj@users.noreply.github.com>, Colin P. McCabe <cmccabe@apache.org>
2025-03-17 14:00:27 -07:00
Nick Guo e9ffe0ba7c
KAFKA-18808 add test to ensure the name=<default> is not equal to default quota (#18966)
see discussion in
[KAFKA-18735](https://issues.apache.org/jira/browse/KAFKA-18735) - the
test should include following check.

1. Using name=<default> does not create default quota
2. the returned entity should have name=<default>
2. the filter `ClientQuotaFilterComponent.ofDefaultEntity` should return
nothing

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 01:57:24 +08:00
ClarkChen ec3c319c35
MINOR: Rewrite unchecked operations in Mock API (#19071)
We encountered unchecked or unsafe operations in
`GroupMetadataManagerTest.java`, `KTableImplTest.java`, and
`ConfigCommandIntegrationTest.java`.

* Rewrite getArgument of invocation in InvocationOnMock API because the
implementation of InvocationOnMock discards type anyway in in
`GroupMetadataManagerTest.java`.

* Remove unchecked annotations for using mock API without variable
assignment in `KTableImplTest.java`.

<img width="1422" alt="Screenshot 2025-03-02 at 8 50 55 AM"
src="https://github.com/user-attachments/assets/10ff1799-ebaa-499c-9acd-ca3b30484e6d"
/>
  
* Follow-up: https://github.com/mockito/mockito/issues/1609

Update on March 2.
* Fix unchecked cast for KTableImpl in `KTableImplTest.java`.
<img width="1259" alt="Screenshot 2025-03-02 at 5 17 47 PM"
src="https://github.com/user-attachments/assets/a5ffa3d7-4897-43ee-9b5f-26337e2560c5"
/>

Update on March 10.
* Use anyMap instead any for unchecked map type issues.
<img width="1691" alt="Screenshot 2025-03-10 at 9 36 38 AM"
src="https://github.com/user-attachments/assets/9aabc595-e7ba-4e04-81f6-f238d42af5a6"
/>

Pass all testing.
<img width="946" alt="Screenshot 2025-03-10 at 10 10 56 AM"
src="https://github.com/user-attachments/assets/793f67ea-09dc-44af-9d6c-de15531e9e72"
/>

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 00:47:34 +08:00
TengYao Chi a6a0ea56d8
KAFKA-17171 Add test cases for `STATIC_BROKER_CONFIG`in kraft mode (#18463)
Given that the `core` module will be separated into other small modules,
this test will not be added to the core module.
Instead, I added it to the `clients-integration-tests` module since it
focuses on the admin client test. The patch should include following test cases.

1. a topic-related static config is added to quorum controller. The
configs from topic creation should include it, but `describeConfigs`
does not.

2. a topic-related static config is added to quorum controller. The
configs from topic creation should include it, and `describeConfigs`
does if admin is using controller.bootstrap

3. a topic-related static config is added to broker. The configs from
topic creation should NOT include it, but `describeConfigs` does.

4. a topic-related static config is added to broker. The configs from
topic creation should NOT include it, and `describeConfigs` does not
also if admin is using controller.bootstrap

for another, the docs of `STATIC_BROKER_CONFIG` should remind the impact of "controller.properties" BTW, those test cases should leverage new test infra, since new test infra allow us to define configs to broker/controller individually.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 00:30:53 +08: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
Bruno Cadonna a7e40b7c5a
KAFKA-18736: Do not send fields if not needed (#19181)
The Streams heartbeat request has some fields that are always sent.
Those are:
- group ID
- member ID
- member epoch
- group instance ID (if static membership is used)

Then it has fields that are only sent when joining:
- topology and topology epoch
- rebalance timeout
- process ID
- endpoint
- client tags

Finally, the assignment is only sent if it changed compared to the last
sent request.

Reviewers: Bill Bejeck <bill@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-03-16 18:08:56 +01:00
Ming-Yen Chung f7d07d62d9
KAFKA-18990 Avoid redundant MetricName creation in BaseQuotaTest#produceUntilThrottled (#19215)
Avoid redundant MetricName creation in BaseQuotaTest#produceUntilThrottled via moving metrics creation out of loop.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-16 21:06:04 +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
Ken Huang 7bff678699
KAFKA-18859 honor the error message of UnregisterBrokerResponse (#19027)
Reviewers: Ismael Juma <ismael@juma.me.uk>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-16 03:06:01 +08:00
TengYao Chi efa6f7ff47
KAFKA-18993 Remove confusing notable change section from upgrade.html (#19212)
Currently, the "Notable changes in 4.0.0" for the client is very confusing. We should remove it.

Reviewers: mingdaoy <mingdaoy@gmail.com>, Luke Chen <showuon@gmail.com>, Ken Huang <s7133700@gmail.com>, David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-16 02:51:49 +08:00
mingdaoy 1d0333f83f
KAFKA-18703 Remove unused class PayloadKeyType (#19129)
the usage of `PayloadKeyType` was removed by
a70e4f95d7

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-16 02:50:20 +08:00
ClarkChen e05b0e68e4
KAFKA-18915 Rewrite AdminClientRebootstrapTest to cover the current scenario (#19187)
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-16 02:35:41 +08:00
Ken Huang 93499df6e9
KAFKA-18924 Running the storage module tests produces a storage/storage.log file (#19147)
Change log4j2 logging directory to `build/kafka-storage-test/`

Reviewers: Sanskar Jhajharia <sjhajharia@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-15 05:20:45 +08:00
Kaushik Raina c32c167e04
KAFKA-18781: Extend RefreshRetriableException related exceptions (#19136)
- Extended derived exceptions described in
[KIP-1050](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=309496816#KIP1050:ConsistenterrorhandlingforTransactions-RefreshRetriableException)
to include the new RefreshRetriableException in base hierarchy.
- Added unit tests to validate the hierarchy of the derived exceptions
in relevant scenarios.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-03-14 09:11:31 -07:00
Ken Huang f50a17fa8d
KAFKA-18606: Flaky test DeleteSegmentsByRetentionTimeTest#executeTieredStorageTest (#18861)
Jira: https://issues.apache.org/jira/browse/KAFKA-18606

This flaky test is caused by
23c459286b,
it modify the default `LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT` from
`Long.MAX_VALUE` to `3600000`(1 hour) and remove the remove the produce
record timestamp
23c459286b (diff-d157da9c40cb386be02d1f917db9e5f6293cdbc82e45a39115bf8629fc19d55cL59).
This test case is testing that data handling after the retention period
(1ms) has expired then delete of related segmant. The old test
https://github.com/apache/kafka/pull/16932 add
`TimeUnit.DAYS.toMillis(1))` makes the third record expire, thus this
test is flaky a lot due to this record.

Reviewers: Jun Rao <jun@confluent.io>
2025-03-14 08:10:36 -07:00
Kondrat Bertalan e3800e02f3
KAFKA-18843: MirrorMaker2 unique workerId (#18994)
Change the worker ID to the combination of the host name, flow and a random UUID to make it unique.

Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2025-03-14 09:49:57 +01:00
Andrew Schofield 5e7445a6d6
KAFKA-17516 Synonyms for client metrics configs (#17264)
This PR brings client metrics configuration resources in line with the
other config resources in terms of handling synonyms and defaults.
Specifically, configs which are not explicitly set take their hard-coded
default values, and these are reported by `kafka-configs.sh --describe`
and `Kafka-client-metrics.sh --describe`. Previously, they were omitted
which means the administrator needed to know the default values.

The ConfigHelper was changed so that the handling of client metrics
configuration matches that of group configuration.

Reviewers: poorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-14 16:05:40 +08:00
Parker Chang 049430e9f9
KAFKA-18927 Remove LATEST_0_11, LATEST_1_0, LATEST_1_1, LATEST_2_0 (#19134)
Revert the following commits:

2ee7e4d22c

d1952e8542

Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-03-14 14:09:31 +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
Gerard Klijs-Nefkens b2a01b2754
MINOR: call the serialize method including headers from the MockProducer (#11144)
Currently when using serializers like the Cloud Event Serializer, we
need to do a work around so it doesn't throw an error. Using the method
taking the headers would prevent this. Since the default implementation
just calls the method without the headers, it's expected to be fully
backwards compatible.

Reviewers: Divij Vaidya <divijvaidya13@gmail.com>
2025-03-13 18:50:29 +01:00
Alieh Saeedi ff785ac251
KAFKA-18651: Add Streams-specific broker configurations (#19176)
This change implements the broker-side configs proposed in KIP-1071.
The configurations implemented by this PR are only those that were specifically aimed to be included in `AK 4.1`.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-03-13 18:05:24 +01:00
Matthias J. Sax 2181ddbb03
KAFKA-18943: Kafka Streams incorrectly commits TX during task revokation (#19164)
Fixes two issues:
 - only commit TX if no revoked tasks need to be committed
 - commit revoked tasks after punctuation triggered

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <sophie@responsive.dev>, Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-03-13 09:37:11 -07: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
Mickael Maison 759fbbba8b
KAFKA-14484: Move UnifiedLog to storage module (#19030)
Rewrite UnifiedLog in Java

Reviewers: Jun Rao <jun@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-13 10:49:55 +01:00
Alieh Saeedi ca834d1174
MINOR: remove incorrect deprecated annotations from internal Streams classes (#19195)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-03-12 17:49:00 -07:00
Mickael Maison 55d65cb3ba
MINOR: Cleanups in CoreUtils (#19175)
Delete unused methods in CoreUtils and switch to Utils.newInstance().

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-12 19:43:30 +01:00
David Arthur 25f8bf82f7
KAFKA-18617 Allow use of ClusterInstance inside BeforeEach (#18662)
Move the ClusterTest setup callback to `BeforeEachCallback` instead of
`BeforeTestExecutionCallback`. This allows the setup to happen slightly
earlier and before any `@BeforeEach` methods are invoked.

We cannot inject ClusterInstance into a `@BeforeEach` method, but we can
still use the constructor injection which allows a `@BeforeEach` fixture
to access the ClusterInstance. This enables tests to do some common
cluster setup (like creating topics) before each test case.

No changes were made to `@AfterEach` or our ClusterTest teardown logic.

Reviewers: Sushant Mahajan <smahajan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-13 01:59:20 +08:00
Dejan Stojadinović e3080684c0
KAFKA-18142 Switch to `com.gradleup.shadow` (#18018)
**Prologue (related PR's):**
-  https://github.com/apache/kafka/pull/16295
-  https://github.com/apache/kafka/pull/17218
-  https://github.com/apache/kafka/pull/16489 

**Action points:**
- switch shadow plugin from `io.github.goooler.shadow` to
`com.gradleup.shadow`
- upgrade plugin version from **8.1.3** to **8.3.5** (release notes:
https://gradleup.com/shadow/changes/#v8-3-5-2024-11-03)

**Rationale:** both previous one and currently used shadow plugins are
now deprecated (in favor of `com.gradleup.shadow`):
- `com.github.johnrengelman.shadow` maintenance was transferred to
`com.gradleup.shadow`:
https://github.com/GradleUp/shadow/tree/8.3.5?tab=readme-ov-file#gradle-shadow
- `io.github.goooler.shadow`: changes are ported to
`com.gradleup.shadow`:
https://github.com/Goooler/shadow?tab=readme-ov-file#gradle-shadow

**Pitfall (to keep in mind):** 
- https://github.com/apache/kafka/pull/15532 

Co-authored-by: Goooler <wangzongler@gmail.com>
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>
2025-03-12 13:49:10 -04:00
Ming-Yen Chung 1a18985cba
KAFKA-18947 Remove unused raftManager in metadataShell (#19169)
* Remove unused `raftManager` in `metadataShell`
* Enhance error message when no snapshot provided.
* Since `raftManager` is removed, make `snapshot` a required argument.

Result when no snapshot is given
```
$ ./bin/kafka-metadata-shell.sh
usage: kafka-metadata-shell [-h] --snapshot SNAPSHOT [command [command ...]]
kafka-metadata-shell: error: argument --snapshot/-s is required
```
```
$ ./bin/kafka-metadata-shell.sh --help
usage: kafka-metadata-shell [-h] --snapshot SNAPSHOT [command [command ...]]

The Apache Kafka metadata shell

positional arguments:
  command                The command to run.

optional arguments:
  -h, --help             show this help message and exit
  --snapshot SNAPSHOT, -s SNAPSHOT
                         The metadata snapshot file to read.
```

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-13 01:48:42 +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
TengYao Chi e1d980a3d1
MINOR: Remove unused ConfigCommandOptions#forceOpt (#19170)
This field is unused, and we should remove it.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-13 00:04:22 +08:00
TaiJuWu a2653281c2
KAFKA-18837: Ensure controller quorum timeouts and backoffs are at least 0 (#18998)
Add validators to the QuorumConfig's configuration definitions.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ken Huang <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>
2025-03-12 16:43:38 +01:00
mingdaoy f0f7382ff7
KAFKA-18422 Adjust Kafka client upgrade path section (#19119)
This patch adds a section about upgrading clients to the upgrade notes.

Reviewers: Ismael Juma <ismael@juma.me.uk>, David Jacot <djacot@confluent.io>
2025-03-12 06:55:08 -07:00
Ming-Yen Chung d964574ab9
KAFKA-18942: Add reviewers to PR body with committer-tools (#19168)
Allow `reviewers.py` to set the "Reviewers:" line in a PR body.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2025-03-12 09:41:05 -04:00
Abhinav Dixit c07c59ad24
KAFKA-18932: Removed usage of partition max bytes from share fetch requests (#19148)
This PR aims to remove the usage of partition max bytes from share fetch
requests. Partition Max Bytes is being defined by
`PartitionMaxBytesStrategy` which was added to the broker as part of PR
https://github.com/apache/kafka/pull/17870

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-03-12 13:19:19 +00:00
Apoorv Mittal f3da8f500e
KAFKA-18936: Fix share fetch when records are larger than max bytes (#19145)
The PR fixes the behaviour when records are fetched which are larger
than `fetch.max.bytes` config.

The usage of `hardMaxBytesLimit` is in ReplicaManager where it decides
whether to fetch a single record or not. The file records get sliced
based on the bytes requested. However, if `hardMaxBytesLimit` is false
then at least one record is fetched and bytes are adjusted accordingly in
`localLog`.

Reviewers: Jun Rao <junrao@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-03-12 09:03:35 +00:00
Ken Huang cf07e6b1f6
KAFKA-18074: Add kafka client compatibility matrix (#18091)
Add new section for Kafka 4.0 compatibility metrics for user, so user
can check server and client in this section.

Reviewers: Divij Vaidya <diviv@amazon.com>, Matthias J. Sax <matthias@confluent.io>
2025-03-11 18:38:37 -07:00
PoAn Yang baa152ced7
KAFKA-18195: Fix Kafka Streams broker compatibility matrix (#18258)
Use "incompatible" instead of an empty cell in Kafka Streams broker
compatibility docs. Also update compatibility matrix for 4.0.0 release.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-03-11 18:38:08 -07:00
Jhen-Yung Hsu a383a954c8
KAFKA-17808: Fix id typo for connector-dlq-adminclient (#19171)
Reviewers: Greg Harris <greg.harris@aiven.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-11 13:37:11 -07:00
David Arthur 701573366f
KAFKA-18933 Add client integration tests module (#19144)
Adds a new ":clients:integration-test" Gradle module. Relocates one
example test from ":core"

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-11 16:36:23 -04:00
David Arthur 903d70d764
MINOR Mark Tls13SelectorTest#testCloseOldestConnection as flaky (#19178)
This test has a flakiness around 7%. It caused two back-to-back failures
on trunk recently.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-11 16:35:38 -04:00
Nick Guo 52061bf4aa
MINOR: Add documentation to describe default quotas for user and client-id (#19184)
Add documentation to describe default quotas for user and client-id, which were previously missing.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-11 23:51:33 +08:00
Bruno Cadonna 811761707c
MINOR: Downgrade log messages of swallowed exception to warnings (#19185)
Exceptions that are caught during closing a task dirty are swallowed.
The corresponding log messages are on error level which is misleading
since the exceptions do not cause any special handling or crash.

This commit downgrades the log messages of swallowed exceptions to
warnings and explains in the log messages that the exceptions are
swallowed.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-03-11 16:43:04 +01:00
Lucas Brutschy 6551e87815
KAFKA-18925: Add streams groups support to Admin.listGroups (#19155)
Add support so that Admin.listGroups can represent
streams groups and their states.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-03-11 15:48:07 +01:00