Commit Graph

5675 Commits

Author SHA1 Message Date
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
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
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
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
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
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
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
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
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
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
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
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
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
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
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
Azhar Ahmed 832dfa36da
KAFKA-18637: Fix max connections per ip and override reconfigurations (#19099)
Reviewers: Christo Lolov <lolovc@amazon.com>, TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-03-10 07:27:48 +00:00
Ken Huang d5413fdb48
KAFKA-17856 Move ConfigCommandTest and ConfigCommandIntegrationTest to tool module (#17767)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 21:05:36 +08: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
ClarkChen 1584d49470
KAFKA-18944 Remove unused setters from ClusterConfig (#19166)
Remove unused `saslServerProperties`, `saslClientProperties`,
`adminClientProperties`, `producerProperties`, and `consumerProperties`
in ClusterConfig.

First, I quickly fixed the unused adminClientProperties, and then I will
move on to https://github.com/apache/kafka/pull/19094 to fix the related
issues.

Pass AdminClientRebootstrapTest
<img width="1398" alt="Screenshot 2025-03-09 at 12 54 57 PM"
src="https://github.com/user-attachments/assets/73c50376-6602-493d-8abd-0eb2bb304114"
/>

Pass ClusterConfigTest
<img width="1117" alt="Screenshot 2025-03-09 at 12 55 28 PM"
src="https://github.com/user-attachments/assets/b4da59da-dfdf-4698-9077-5086854360ab"
/>

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 17:49:28 +08:00
ClarkChen 2a0dbd8e0b
KAFKA-18909 Move DynamicThreadPool to server module (#19081)
* Add `DynamicThreadPool.java` to the server module.
* Remove the old DynamicThreadPool object in the `DynamicBrokerConfig.scala`.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 17:42:51 +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
Dániel Urbán 40db001588
KAFKA-18929: Log a warning when time based segment delete is blocked by a future timestamp (#19137)
When producers send future timestamps, time retention based log segments
may get blocked from removal for an extended period of time. Log
cleaning should should warn in the logs when this scenario occurs.

Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2025-03-07 14:31:22 +01:00
ClarkChen 870db5d811
KAFKA-18915: Migrate AdminClientRebootstrapTest to use new test infra (#19094)
Migrate AdminClientRebootstrapTest to the new test infra and remove the
old Scala test.

Reviewers: TengYao Chi <kitingiao@gmail.com>, David Arthur <mumrah@gmail.com>
2025-03-06 16:05:51 -05:00
Andrew Schofield 1da30bdedf
KAFKA-18900: Experimental share consumer acknowledge mode config (#19113)
User testing of the `KafkaShareConsumer` interface has revealed some
areas which confuse people. One of these is that way that it decides
whether you want to use implicit or explicit acknowledgement of records
by observing which calls the application issues. We are taking the
opportunity to refine the interface before it is finalised.

This PR introduces an experimental configuration called
`internal.share.acknowledgement.mode` which can be used to make the
application declare which kind of acknowledgement it wishes to use. We
plan to try out the configuration, assess whether it has helped, and
then create a proper consumer configuration that makes this area better.
That would require a lot of change in the tests, which explains why this
initial PR only has a small number of tests.

Reviewers: David Arthur <mumrah@gmail.com>
2025-03-06 17:57:11 +00:00
Ken Huang 041d8019d6
KAFKA-18910 Remove kafka.utils.json (#19112)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-06 14:11:20 +08: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
Xuan-Zhang Gong 18eca0229d
KAFKA-18882 Remove BaseKey, TxnKey, and UnknownKey (#19054)
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-05 21:16:18 +08:00
Lan Ding 69ff5d1e70
KAFKA-18817: ShareGroupHeartbeat and ShareGroupDescribe API must check topic describe (#19083)
Reviewers: Christo Lolov <lolovc@amazon.com>
2025-03-05 11:25:08 +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
Logan Zhu 011f256c86
KAFKA-18886 add behavior change of CreateTopicPolicy and AlterConfigPolicy to zk2kraft (#19087)
1. Updated JavaDoc to reflect that CreateTopicPolicy and AlterConfigPolicy run on the controller in KRaft mode.
2. Modified Behavioral Change Reference in the HTML docs to include this change.
3. add warning message to KafkaConfig if the config of broker node has policy configs 


Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-05 15:15:03 +08:00
co63oc e4ece37dbf
Fix typos in multiple files (#19086)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-04 16:05:51 +00:00
Apoorv Mittal c1fc59fc23
KAFKA-18918: Correcting releasing of locks on exception (#19091)
The PR corrects the way the locks are released on exception. As
`partitionsAcquired` can be a reference to `topicPartitionData`, hence
the locks should released prior clearing `partitionsAcquired`.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-03-04 16:04:45 +00:00
David Jacot 1df4a42b40
KAFKA-18916; Resolved regular expressions must update the group by topics data structure (#19088)
When regular expressions are resolved, they do not update the group by
topics data structure. Hence, topic changes (e.g. deletion) do not
trigger a rebalance of the group.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-03-04 06:31:08 -08:00
Nick Guo 101e15bb1c
KAFKA-18867 add tests to describe topic configs with empty name (#19075)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-04 14:56:25 +08:00
Mahsa Seifikar 2154e55abf
MINOR: Prevent broker fencing by adjusting resendExponentialBackoff in BrokerLifecycleManager (#19061)
This PR reduces `maxInterval` for `resendExponentialBackoff` in
`BrokerLifecycleManager` class from `broker.session.timeout.ms` to half
of its value. Setting `maxInterval` to `broker.session.timeout.ms`
caused brokers to be fenced if a resend attempt occurred near the
timeout threshold, leading to unnecessary broker fencing.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-03-03 12:03:15 -08:00
Apoorv Mittal a6c53d0c37
KAFKA-18878: Added share session cache and delayed share fetch metrics (KIP-1103) (#19059)
The PR implements the ShareSessionCache and DelayedShareFetchMetrics as
defined in KIP-1103.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-03 16:44:34 +00:00
Lucas Brutschy a04dd21f26
KAFKA-18613: Auto-creation of internal topics in streams group heartbeat (#18981)
Implements auto-topic creation when handling the streams group
heartbeat.

Inside KafkaApis, the handler for streamsGroupHeartbeat uses the result
of the streams group heartbeat inside the group coordinator to attempt
to create all missing internal topics using AutoTopicCreationManager.
CREATE TOPIC ACLs are checked. The unit tests class
AutoTopicCreationManagerTest is brought back (it was recently deleted
during a ZK removal PR), but testing only the kraft-related
functionality.

Reviewers: Bruno Cadonna <bruno@confluent.io>

### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation 
- [ ] Verify test coverage and CI build status
- [ ] Verify documentation (including upgrade notes)
2025-03-03 08:48:00 +01:00
Xuan-Zhang Gong ceac4f0a1d
KAFKA-18880 Remove kafka.cluster.Broker and BrokerEndPointNotAvailableException (#19047)
Remove kafka.cluster.Broker and BrokerEndPointNotAvailableException as they were used by zk path.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-02 10:54:32 +08:00
TengYao Chi e0c77140b2
KAFKA-17039 KIP-919 supports for unregisterBroker (#19063)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 23:55:35 +08:00
Nick Guo 98bb79e732
KAFKA-17981 add Integration test for ConfigCommand to add config `key=[val1,val2]` (#17771)
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 13:15:25 +08:00
Apoorv Mittal 8cf969e00a
KAFKA-18734: Implemented share partition metrics (KIP-1103) (#19045)
The PR implements the SharePartitionMetrics as defined in KIP-1103, with
one change. The metric `FetchLockRatio` is defined as `Meter` in KIP but
is implemented as `HIstogram`. There was a discussion about same on
KIP-1103 discussion where we thought that `FetchLockRatio` is
pre-aggregated but while implemeting the rate from `Meter` can go above
100 as `Meter` defines rate per time period. Hence it makes more sense
to implement metric `FetchLockRatio` as `Histogram`.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-28 14:22:27 +00:00
Apoorv Mittal 8b605bd362
MINOR: Removing share partition manager flaky annotation (#19053)
There isn't any flaky test for SharePartitionManager in last 7 days, removing flaky annotation.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-28 08:49:59 +00:00
Dongnuo Lyu 36f19057e1
KAFKA-18813: ConsumerGroupHeartbeat API and ConsumerGroupDescribe API must check topic describe (#18989)
This patch filters out the topic describe unauthorized topics from the
ConsumerGroupHeartbeat and ConsumerGroupDescribe response.

In ConsumerGroupHeartbeat, 
- if the request has `subscribedTopicNames` set, we directly check the
authz in `KafkaApis` and return a topic auth failure in the response if
any of the topics is denied.
- Otherwise, we check the authz only if a regex refresh is triggered and
we do it based on the acl of the consumer that triggered the refresh. If
any of the topic is denied, we filter it out from the resolved
subscription.

In ConsumerGroupDescribe, we check the authz of the coordinator
response. If any of the topic in the group is denied, we remove the
described info and add a topic auth failure to the described group.
(similar to the group auth failure)

Reviewers: David Jacot <djacot@confluent.io>, Lianet Magrans
<lmagrans@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>,
Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu <tjwu1217@gmail.com>,
TengYao Chi <kitingiao@gmail.com>
2025-02-26 13:05:36 -05:00
Lucas Brutschy cb7c54ccd3
KAFKA-18614, KAFKA-18613: Add streams group request plumbing (#18979)
This change implements the basic RPC handling StreamsGroupHeartbeat and
StreamsGroupDescribe. This includes:
 - Adding an option to enable streams groups on the broker
- Passing describe and heartbeats to the right shard of the group
coordinator
- The handler inside the GroupMetadatManager for StreamsGroupDescribe is
fairly trivial, and is included directly in this PR.
- The handler for StreamsGroupHeartbeat is complex and not included in
this PR yet. Instead, a UnsupportedOperationException is thrown.
However, the interface is already defined: The result of a
streamsGroupHeartbeat is a response, together with a list of internal
topics to be created.

The heartbeat implementation inside the `GroupMetadataManager`, which
actually implements the assignment / reconciliation logic, will come in
a follow-up PR. Also, automatic creation of internal topics will be
created in a follow-up PR.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-26 16:33:26 +01:00
Abhinav Dixit 4b5a16bf6f
KAFKA-18757: Create full-function SimpleAssignor to match KIP-932 description (#18864)
### About
The current `SimpleAssignor` in AK assigned all subscribed topic
partitions to all the share group members. This does not match the
description given in
[KIP-932](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255070434#KIP932:QueuesforKafka-TheSimpleAssignor).
Here are the rules as mentioned in the KIP by which the assignment
should happen. We have changed the step 3 implementation here due to the
reasons
[described](https://github.com/apache/kafka/pull/18864#issuecomment-2659266502)
-

1. The assignor hashes the member IDs of the members and maps the
partitions assigned to the members based on the hash. This gives
approximately even balance.
2. If any partitions were not assigned any members by (1) and do not
have members already assigned in the current assignment, members are
assigned round-robin until each partition has at least one member
assigned to it.
3. We combine the current and new assignment. (Original rule - If any
partitions were assigned members by (1) and also have members in the
current assignment assigned by (2), the members assigned by (2) are
removed.)

### Tests
The added code has been verified with unit tests and the already present
integration tests.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, TaiJuWu <tjwu1217@gmail.com>
2025-02-26 11:02:23 +00:00
José Armando García Sancio 4a8a0637e0
KAFKA-18723; Better handle invalid records during replication (#18852)
For the KRaft implementation there is a race between the network thread,
which read bytes in the log segments, and the KRaft driver thread, which
truncates the log and appends records to the log. This race can cause
the network thread to send corrupted records or inconsistent records.
The corrupted records case is handle by catching and logging the
CorruptRecordException. The inconsistent records case is handle by only
appending record batches who's partition leader epoch is less than or
equal to the fetching replica's epoch and the epoch didn't change
between the request and response.

For the ISR implementation there is also a race between the network
thread and the replica fetcher thread, which truncates the log and
appends records to the log. This race can cause the network thread send
corrupted records or inconsistent records. The replica fetcher thread
already handles the corrupted record case. The inconsistent records case
is handle by only appending record batches who's partition leader epoch
is less than or equal to the leader epoch in the FETCH request.

Reviewers: Jun Rao <junrao@apache.org>, Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@apache.org>
2025-02-25 20:09:19 -05:00
Apoorv Mittal df5839a9f4
KAFKA-17351: Improved handling of compacted topics in share partition (2/N) (#19010)
The PR handles fetch for `compacted` topics. The fix was required only
when complete batch disappears from the topic log, and same batch is
marked re-available in Share Partition state cache. Subsequent log reads
will not result the disappeared batch in read response hence respective
batch will be left as available in the state cache.

The PR checks for the first fetched/read batch base offset and if it's
greater than the position from where the read occurred (fetch offset)
then if there exists any `available` batches in the state cache then
they will be archived.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-02-25 14:11:39 +00:00
xijiu 1edc30bf30
KAFKA-17836 Move RackAwareTest to server module (#19021)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 18:15:34 +08:00