Commit Graph

14999 Commits

Author SHA1 Message Date
Luke Chen 8026d6b3e8 KAFKA-18230: Handle not controller or not leader error in admin client (#18165)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-04 16:52:11 +01:00
David Jacot fe258c74ef MINOR: Fix TestBounce sys test (#18798)
```
================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.12.0
session_id:       2025-02-04--005
run time:         4 minutes 0.023 seconds
tests run:        4
passed:           4
flaky:            0
failed:           0
ignored:          0
================================================================================
```

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-02-04 14:49:34 +01:00
David Jacot ff798dcdf1 MINOR: Fix PerformanceService sys test (#18797)
This patch fixes the PerformanceService system test which was still using ZK.

```
================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.12.0
session_id:       2025-02-04--003
run time:         1 minute 42.629 seconds
tests run:        4
passed:           4                                                                                                                                                                         flaky:            0
failed:           0                                                                                                                                                                         ignored:          0
================================================================================
```

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-04 14:45:15 +01:00
David Jacot 593c99b013 MINOR: Fix Benchmark sys tests (#18796)
This patch fixes the Benchmark system tests. We misconfigured the quorum in bc7b87001b.

```
================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.12.0
session_id:       2025-02-04--001
run time:         57 minutes 27.169 seconds
tests run:        62
passed:           62
flaky:            0
failed:           0
ignored:          0
================================================================================
```

Reviewers: PoAn Yang <payang@apache.org>, Christo Lolov <lolovc@amazon.com>
2025-02-04 14:45:07 +01:00
Matthias J. Sax 53aa8dedea MINOR: fix NPE in KS `Topology` for new `AutoOffsetReset` (#18780)
Introduced via KIP-1106.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-02-03 17:30:49 -08:00
Justine Olshan d119864f16 KAFKA-18654 [1/2]: Transaction Version 2 performance regression due to early return (#18720)
https://issues.apache.org/jira/browse/KAFKA-18575 solved a critical race condition by returning with CONCURRENT_TRANSACTIONS early when the transaction was still completing.
In testing, it was discovered that this early return could cause performance regressions.

Prior to KIP-890 the addpartitions call was a separate call from the producer. There was a previous change https://issues.apache.org/jira/browse/KAFKA-5477 that decreased the retry backoff to 20ms. With KIP-890 and making the call through the produce path, we go back to the default retry backoff which takes longer. Prior to 18575 we introduce a slight delay when sending to the coordinator, so prior to 18575, we are less likely to return quickly and get stuck in this backoff. However, based on results from produce benchmarks, we can still run into the default backoff in some scenarios.

This PR reverts KAFKA-18575, and doesn't return early and wait until the coordinator for checking if a transaction is ongoing. Instead, it will fix the handling with the verification guard so we don't hit the edge condition.

Also cleans up some of the verification text that was unclear.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Artem Livshits <alivshits@confluent.io>
2025-02-03 15:25:51 -08:00
Ken Huang 85bd059559 KAFKA-18545: Remove Zookeeper logic from LogManager (#18592)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Mickael Maison <mickael.maison@gmail.com>
2025-02-03 17:17:24 +00:00
David Jacot 00630ce166
KAFKA-18672; CoordinatorRecordSerde must validate value version (4.0) (#18786)
CoordinatorRecordSerde does not validate the version of the value to check whether the version is supported by the current version of the software. This is problematic if a future and unsupported version of the record is read by an older version of the software because it would misinterpret the bytes. Hence CoordinatorRecordSerde must throw an error if the version is unknown. This is also consistent with the handling in the old coordinator.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2025-02-03 08:18:58 -08:00
Dongnuo Lyu dd40efba2c KAFKA-18655: Implement the consumer group size counter with scheduled task (#18717)
During testing we discovered that the empty group count is not updated in group conversion, but when the new group is transition to other state, the empty group count is decremented. This could result in negative empty group count.

We can have a new consumer group count implementation that follows the pattern we did for the classic group count. The timeout task periodically refreshes the metrics based on the current groups soft state.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2025-02-03 10:53:19 -05:00
Ken Huang b49c573d1b KAFKA-18685: Cleanup DynamicLogConfig constructor (#18764)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Christo Lolov <lolovc@amazon.com>
2025-02-03 15:40:18 +00:00
PoAn Yang 9fb43fc69a KAFKA-18676; Update Benchmark system tests (#18785)
Update `benchmark_test.py` to use KRaft.

```
> TC_PATHS="tests/kafkatest/benchmarks/core/benchmark_test.py" /bin/bash tests/docker/run_tests.sh

================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.12.0
session_id:       2025-02-03--001
run time:         96 minutes 48.900 seconds
tests run:        120
passed:           120
flaky:            0
failed:           0
ignored:          0
================================================================================
```

Reviewers: David Jacot <djacot@confluent.io>
2025-02-03 14:42:44 +01:00
PoAn Yang 844d2c0383 KAFKA-17631 Convert SaslApiVersionsRequestTest to kraft (#18330)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-03 21:02:26 +08:00
Jhen-Yung Hsu a9a535d741 MINOR: Remove the test for ZooKeeper metrics used by ZooKeeperClient (#18775)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-03 20:16:01 +08:00
PoAn Yang 2c7a84170f MINOR: add removed controller metrics to zk2kraft (#18654)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-03 20:15:17 +08:00
PoAn Yang 67b29e162b KAFKA-18678 Update TestVerifiableProducer system test (#18768)
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-03 11:14:14 +01:00
Ming-Yen Chung 563d729a57 KAFKA-18674 Document the incompatible changes in parsing --bootstrap-server (#18751)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-03 13:57:54 +08:00
Ismael Juma 0e01d38a27 KAFKA-18659: librdkafka compressed produce fails unless api versions returns produce v0 (#18727)
Return produce v0-v2 as supported versions in `ApiVersionsResponse`, but disable support
for it everywhere else.

Since clients pick the highest supported version by both client and broker during version
negotiation, this solves the problem with minimal tech debt (even though it's not ideal that
`ApiVersionsResponse` becomes inconsistent with the actual protocol support).

Add one test for the socket server handling (in `ProcessorTest`) and one test for the
client behavior (in `ProduceRequestTest`). Adjust a couple of api versions tests to verify
the new behavior.

Finally, include a few clean-ups in `ApiKeys`, `Protocol`, `ProduceRequest`,
`ProduceRequestTest` and `BrokerApiVersionsCommandTest`.

Reference to related librdkafka issue:
https://github.com/confluentinc/librdkafka/issues/4956

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2025-02-01 16:19:46 -08:00
Lianet Magrans fc3dca4ed0 Revert "KAFKA-17182: Consumer fetch sessions are evicted too quickly with AsyncKafkaConsumer (#17700)"
This reverts commit 9c02072c4f.
2025-01-31 17:25:17 -05:00
Justine Olshan 685ce7b354
KAFKA-18660: Transactions Version 2 doesn't handle epoch overflow correctly (#18730) (#18758)
Fixed the typo that used the wrong producer ID and epoch when returning so that we handle epoch overflow correctly.

We also had to rearrange the concurrent transaction handling so that we don't self-fence when we start the new transaction with the new producer ID.

I also tested this with a modified version of the code where epoch overflow happens on the first epoch bump (every request has a new producer id)

Reviewers: Artem Livshits <alivshits@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2025-01-31 09:23:31 -08:00
David Jacot a2d2556297 KAFKA-18320; Ensure that assignors are at the right place (#18750)
The full class name of the assignors if part of our public api. Hence, we should ensure that they are not changed by mistake. This patch adds a unit test verifying them.

Reviewers: Sean Quah <squah@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2025-01-31 16:51:57 +01:00
Kirk True 9c02072c4f KAFKA-17182: Consumer fetch sessions are evicted too quickly with AsyncKafkaConsumer (#17700)
This change reduces fetch session cache evictions on the broker for AsyncKafkaConsumer by altering its logic to determine which partitions it includes in fetch requests.

Background
Consumer implementations fetch data from the cluster and temporarily buffer it in memory until the user next calls Consumer.poll(). When a fetch request is being generated, partitions that already have buffered data are not included in the fetch request.

The ClassicKafkaConsumer performs much of its fetch logic and network I/O in the application thread. On poll(), if there is any locally-buffered data, the ClassicKafkaConsumer does not fetch any new data and simply returns the buffered data to the user from poll().

On the other hand, the AsyncKafkaConsumer consumer splits its logic and network I/O between two threads, which results in a potential race condition during fetch. The AsyncKafkaConsumer also checks for buffered data on its application thread. If it finds there is none, it signals the background thread to create a fetch request. However, it's possible for the background thread to receive data from a previous fetch and buffer it before the fetch request logic starts. When that occurs, as the background thread creates a new fetch request, it skips any buffered data, which has the unintended result that those partitions get added to the fetch request's "to remove" set. This signals to the broker to remove those partitions from its internal cache.

This issue is technically possible in the ClassicKafkaConsumer too, since the heartbeat thread performs network I/O in addition to the application thread. However, because of the frequency at which the AsyncKafkaConsumer's background thread runs, it is ~100x more likely to happen.

Options
The core decision is: what should the background thread do if it is asked to create a fetch request and it discovers there's buffered data. There were multiple proposals to address this issue in the AsyncKafkaConsumer. Among them are:

The background thread should omit buffered partitions from the fetch request as before (this is the existing behavior)
The background thread should skip the fetch request generation entirely if there are any buffered partitions
The background thread should include buffered partitions in the fetch request, but use a small “max bytes” value
The background thread should skip fetching from the nodes that have buffered partitions
Option 4 won out. The change is localized to AbstractFetch where the basic idea is to skip fetch requests to a given node if that node is the leader for buffered data. By preventing a fetch request from being sent to that node, it won't have any "holes" where the buffered partitions should be.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Jun Rao <junrao@gmail.com>
2025-01-31 09:12:25 -05:00
David Jacot 7eed1fc823 KAFKA-18146; tests/kafkatest/tests/core/upgrade_test.py needs to be re-added as KRaft (#18766)
This patch renames kraft_upgrade_test.py to upgrade_test.py. This is enough to cover the old upgrade/downgrade tests.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-31 15:08:47 +01:00
TengYao Chi 78057eb7f7 KAFKA-18677; Update ConsoleConsumerTest system test (#18763)
This patch converts the ConsoleConsumerTest system test to only use KRaft.

Reviewers: David Jacot <djacot@confluent.io>
2025-01-31 12:20:17 +01:00
Luke Chen 072c32514a MINOR: Clean up for sasl endpoints (#18519)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-31 09:27:43 +01:00
Lianet Magrans 1f18753a1f
KAFKA-18034: CommitRequestManager should fail pending requests on fatal coordinator errors (#18754)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-30 16:55:51 -05:00
Joao Pedro Fonseca Dantas 37503cf42e MINOR: remove close from contextual processors javadoc
Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-01-30 11:04:07 -08:00
Jeff Kim 58a7fab483 MINOR: prevent exception from HdrHistogram (#18674)
HdrHistogram can throw an exception if the recorded value is greater than a configured limit. Expand the ceiling from per-metric to all invocations.

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-30 10:21:45 -05:00
Almog Gavra 9be3cb3d4b MINOR: fix typo in HTML docs (#18742)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-30 12:12:09 +01:00
Mehari Beyene 244be63ddb KAFKA-18570: Update documentation to add remainingLogsToRecover, remainingSegmentsToRecover and LogDirectoryOffline metrics (#18731)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-30 12:12:09 +01:00
PoAn Yang a2d88f881c KAFKA-18441: Fix flaky KafkaAdminClientTest#testAdminClientApisAuthenticationFailure (#18735)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-30 10:47:30 +00:00
TengYao Chi 90573b4b53 KAFKA-18569: New consumer close may wait on unneeded FindCoordinator (#18590)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Kirk True <ktrue@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-29 14:18:59 -05:00
Joao Pedro Fonseca Dantas 4eb35a435d KAFKA-16339: Add Kafka Streams migrating guide from transform to process (#18314)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-01-29 11:11:20 -08:00
PoAn Yang ffb99a45f0 KAFKA-18383 Remove reserved.broker.max.id and broker.id.generation.enable (#18478)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-30 02:55:37 +08:00
Calvin Liu c61cf07e48 KAFKA-18662: Return CONCURRENT_TRANSACTIONS on produce request in TV2 (#18733)
While testing, it was found that the not_enough_replicas error was super common and could be easily confused. Since we are already bumping the request, we can signify that the produce request may return this error and new clients can handle it 

(Note, the java client should be able to handle this already as a retriable error, but other client libraries may need to implement this change)

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-01-29 10:17:20 -08:00
Ismael Juma e6cf82bb6c KAFKA-18646: Null records in fetch response breaks librdkafka (#18726)
Ensure we always return empty records (including cases where an error is returned).
We also remove `nullable` from `records` since it is effectively expected to be
non-null by a large percentage of clients in the wild.

This behavior regressed in fe56fc9 (KAFKA-18269). Empty records were
previously set via `FetchResponse.recordsOrFail(partitionData)` in the
now-removed `maybeConvertFetchedData` method.

Added an integration test that fails without this fix and also update many
tests to set `records` to `empty` instead of leaving them as `null`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2025-01-29 07:17:26 -08:00
TengYao Chi 52280cdb55 KAFKA-18619: New consumer topic metadata events should set requireMetadata flag (#18668)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-01-29 08:39:54 -05:00
Ismael Juma 7139bf0838 KAFKA-18648: Add back support for metadata version 0-3 (#18716)
During testing, we identified that kafka-python (and aiokafka) relies on metadata request v0 and
hence we need to add these back to comply with the premise of KIP-896 - i.e. it should not
break the clients listed within it.

I reverted the changes from #18218 related to the removal of metadata versions 0-3.

I will submit a separate PR to undeprecate these API versions on the relevant 3.x branches.

kafka-python (and aiokafka) work correctly (produce & consume) with this change on
top of the 4.0 branch.

Reviewers: David Arthur <mumrah@gmail.com>
2025-01-28 18:46:33 -08:00
TaiJuWu aeb9acc7ef KAFKA-18528: MultipleListenersWithSameSecurityProtocolBaseTest and GssapiAuthenticationTest should run for async consumer (#18555)
Reviewers: Kirk True <ktrue@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2025-01-28 10:05:11 -05:00
David Arthur 7f9e3a3500 KAFKA-17587 Refactor test infrastructure (#18602)
This patch reorganizes our test infrastructure into three Gradle modules:

":test-common:test-common-internal-api" is now a minimal dependency which exposes interfaces and annotations only. It has one project dependency on server-common to expose commonly used data classes (MetadataVersion, Feature, etc). Since this pulls in server-common, this module is Java 17+. It cannot be used by ":clients" or other Java 11 modules.

":test-common:test-common-util" includes the auto-quarantined JUnit extension. The @Flaky annotation has been moved here. Since this module has no project dependencies, we can add it to the Java 11 list so that ":clients" and others can utilize the @Flaky annotation

":test-common:test-common-runtime" now includes all of the test infrastructure code (TestKitNodes, etc). This module carries heavy dependencies (core, etc) and so it should not normally be included as a compile-time dependency.

In addition to this reorganization, this patch leverages JUnit SPI service discovery so that modules can utilize the integration test framework without depending on ":core". This will allow us to start moving integration tests out of core and into the appropriate sub-module. This is done by adding ":test-common:test-common-runtime" as a testRuntimeOnly dependency rather than as a testImplementation dependency. A trivial example was added to QuorumControllerTest to illustrate this.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-27 15:29:02 -05:00
Martin Sillence 54c3310032 KAFKA-17792: Efficiently parse decimals with large exponents in Connect Values (#17510)
Reviewers: Greg Harris <greg.harris@aiven.io>, Mickael Maison <mickael.maison@gmail.com>
2025-01-27 09:06:28 -08:00
Clay Johnson 9aa21cc691 KAFKA-18454 Publish build scans to develocity.apache.org (#18539)
This patch includes some maintenance updates for Develocity.

* Publish build scans to develocity.apache.org
* Update Develocity Gradle plugin to to 3.19
* Use `DEVELOCITY_ACCESS_KEY` to authenticate to `develocity.apache.org`

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2025-01-27 10:16:40 -05:00
Chung, Ming-Yen 8d5bd25847 KAFKA-18631 Remove ZkConfigs (#18693)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-26 04:43:48 +08:00
Ken Huang 3ee4508bb6 KAFKA-18533 Remove KafkaConfig zookeeper related logic (#18547)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-26 04:29:08 +08:00
TengYao Chi 5f3b175b64 KAFKA-18630: Clean ReplicaManagerBuilder (#18687)
Reviewers: Christo Lolov <lolovc@amazon.com>
2025-01-24 17:36:56 +00:00
TengYao Chi f1b36343ba KAFKA-18592 Cleanup ReplicaManager (#18621)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-24 17:36:48 +00:00
Ken Huang 9c996f7598 KAFKA-18474: Remove zkBroker listener (#18477)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>
2025-01-24 06:09:08 -08:00
Chung, Ming-Yen 448b93bccd KAFKA-18599: Remove Optional wrapping for forwardingManager in ApiVersionManager (#18630)
`forwardingManager` is always present now.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-24 06:08:56 -08:00
Sean Quah 3fe33f8ad6 KAFKA-18484 [2/2]; Handle exceptions during coordinator unload (#18667)
Ensure that unloading a coordinator always succeeds. Previously, we have
guarded against exceptions from DeferredEvent completions. All that
remains is handling exceptions from the onUnloaded() method of the
coordinator state machine.

Reviewers: David Jacot <djacot@confluent.io>
2025-01-23 17:16:21 +01:00
David Jacot c5a51aba18 KAFKA-18486; [1/2] Update LocalLeaderEndPointTest (#18666)
This patch is a first step towards removing `ReplicaManager#becomeLeaderOrFollower`. It updates the `LocalLeaderEndPointTest` tests.

Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>
2025-01-23 10:53:10 +01:00
Ken Huang 6a6e62854f MINOR: add the removed topic metrics to zk2kraft (#18658)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-23 15:05:37 +08:00