Commit Graph

15343 Commits

Author SHA1 Message Date
Lianet Magrans c56c9faee2
KAFKA-18813: [2/N] Client support for TopicAuthException in HB path (#18986)
Reviewers: David Jacot <djacot@confluent.io>
2025-02-21 08:45:20 -05:00
TengYao Chi 767a62ade6
KAFKA-18737 KafkaDockerWrapper setup functions fails due to storage format command (#18844)
The current Docker Hub documentation for Kafka is based on the use of static voters. Since Kafka 4.0 utilizes dynamic voters, users following the doc of docker hub may encounter unexpected behavior. Due to the limited time available for the 4.0.0 release, a simple and quick solution is to revert to using static voters within the Docker image. This can be achieved by adding a configuration file with static voter definitions to the kafka/docker folder, keeping it separate from the main kafka/config directory. This approach allows us to encourage the use of dynamic voters in typical deployments while maintaining compatibility within the Docker image.

Reviewers: Vedarth Sharma <142404391+VedarthConfluent@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 20:43:41 +08:00
David Jacot 2124511431
MINOR: Rearrange configs in GroupCoordinatorConfigs (#18970)
I was looking into GroupCoordinatorConfigs to review configurations that
we will ship with Apache Kafka 4.0. I found out that it was pretty
disorganised. This patch cleans up the format and re-groups the
configurations which are related.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 13:20:58 +01:00
Sushant Mahajan c2cb543a1e
KAFKA-18629: Delete share group state RPC group coordinator impl. [3/N] (#18848)
* In this PR, we have added GC side impl to call the delete state share
coord RPC using the persister.
* We will be using the existing `GroupCoordinatorService.deleteGroups`.
The logic will be modified as follows:
* After sanitization, we will call a new
`runtime.scheduleWriteOperation` (not read for consistency) with
callback `GroupCoordinatorShard.sharePartitions`. This will return a Map
of share partitions of the groups which are of SHARE type. We need to
pass all groups as WE CANNOT DETERMINE the type of the group in the
service class.
* Then using the map we will create requests which could be passed to
the persister and make the appropriate calls.
* Once this future completes, we will continue with the existing flow of
group deletion.
* If the group under inspection is not share group - the read callback
should return an empty map.
* Tests have been added wherever applicable.

Reviewers: David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-21 12:13:16 +00:00
TengYao Chi d31cbf59de
KAFKA-18831 Migrating to log4j2 introduce behavior changes of adjusting level dynamically (#18969)
fix the following behavior changes.

1) in log4j 1, users can't change the logger by parent if the logger is declared by properties explicitly. For example, `org.apache.kafka.controller` has level explicitly in the properties. Hence, we can't use "org.apache.kafka=INFO" to change the level of `org.apache.kafka.controller` to INFO. By contrast, log4j2 allows us to change all child loggers by the parent logger.

2) in log4j2, we can change the level of root to impact all loggers' level. By contrast, log4j 1 can't. 

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 16:12:58 +08:00
Matthias J. Sax acea35ddf3
MINOR: cleanup SinkNode generics (#18975)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-02-20 17:47:39 -08:00
TengYao Chi 709bfc506a
KAFKA-18641: AsyncKafkaConsumer could lose records with auto offset commit (#18737)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Jun Rao <jun@confluent.io>, Kirk True <ktrue@confluent.io>
2025-02-20 12:11:01 -05:00
Calvin Liu 1eecd02ce8
MINOR: Deflake EligibleLeaderReplicasIntegrationTest (#18923)
Make sure to give enough time for the partition ISR updates.

Reviewers: David Jacot <djacot@confluent.io>
2025-02-20 05:14:15 -08:00
Sushant Mahajan c89fd2bff6
KAFKA-18828: Update share group metrics per new init and call mechanism. (#18962)
* Due to recent changes in the way group count metrics are initialized
and updated, the current share group count code has become obsolete as
well as non-functional.
* The update method for the share group count which should be called
from `ShareGroup` cannot be called either. This is because the
constructor has been changed to NOT accept the
`GroupCoordinatorShardMetrics` ref.
* In this PR, we remedy the situation by bringing share group count code
at par with consumer and streams groups.
* Additionally the metric name for share groups with group state
attributes was not aligned with streams and consumer groups as mentioned
in https://github.com/apache/kafka/pull/17011#discussion_r1960309578.
This PR aligns them too.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-20 10:23:37 +00:00
Ken Huang eda8fc84ae
KAFKA-16918 TestUtils#assertFutureThrows should use future.get with timeout (#18891)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Luke Chen <showuon@gmail.com>, Parker Chang <45290853+Parkerhiphop@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-20 07:22:31 +08:00
Matthias J. Sax 9f23b25f6e
MINOR: fix Kafka Streams "smoke test" pass criteria (#18835)
Reviewers: Bill Bejeck <bill@confluent.io>, Bruno Cadonna <bruno@confluent.io>
2025-02-19 14:33:31 -08: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
Ismael Juma 3a59a526d9
MIINOR: Remove redundant quorum parameter from *AdminIntegrationTest classes (#18965)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-02-19 15:57:47 -05:00
David Arthur 7d628401e9
KAFKA-18791 Set default commit to PR title and description [2/n] (#18967)
Reviewers: Justine Olshan <jolshan@confluent.io>
2025-02-19 14:46:53 -05: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
Shivsundar R 3603c8fe35
KAFKA-18829: Added check before converting to IMPLICIT mode (#18964)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-19 17:34:28 +00:00
Ismael Juma 6aab304542
MINOR: Update upgrade notes for 4.0.0 (#18960)
Details:
1. Upgrades to 4.0.x are only supported from 3.3.x and for kraft mode clusters
2. Add rolling upgrade instructions for 4.0.x
3. Clarify the message for zk to kraft migrations
4. Remove all the upgrade instructions for older versions (they can still be found in the upgrade notes for older releases)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2025-02-19 09:32:50 -08:00
Kaushik Raina 469c55cf02
Add TransactionAbortableException and Timeout Exception handling instruction in docs (#18942)
Add instruction for handing TransactionAbortableException and TimeoutException at application side.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-02-19 09:15:47 -08:00
David Arthur 9b29e91218
KAFKA-18791 Enable new asf.yaml parser [1/n] (#18955)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 10:48:23 -05: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
ShivsundarR a6a588fbed
KAFKA-18198: Added check to prevent acknowledgements on initial ShareFetchRequest. (#18944)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-19 10:49:58 +00:00
Matthias J. Sax 900d81b345
MINOR: cleanup top level class JavaDocs for main interfaces of Kafka Streams DSL (4/N) (#18884)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-18 16:22:18 -08:00
Matthias J. Sax 490ba8a8a3
MINOR: cleanup top level class JavaDocs for main interfaces of Kafka Streams DSL (3/N) (#18883)
Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-18 16:21:52 -08:00
Matthias J. Sax d6146644b3
MINOR: verify that internal StreamsConfig members are not public (#18840)
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2025-02-18 16:18:29 -08:00
xijiu 4c4458c17a
KAFKA-18799 Remove AdminUtils (#18946)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 06:25:43 +08:00
David Jacot d847d5c464
MINOR: Tweak default group coordinator config & upgrade notes (#18948)
This patch changes the default value of `group.coordinator.threads` to `4` and sets it priority to `HIGH`. This change makes it consistent with how we handle `num.network.threads` and `num.io.threads`. The patch also tweaks the upgrade notes.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-02-18 20:05:52 +01:00
kevin-wu24 929a90da50
KAFKA-18667 Add replication system test case for combined broker + controller failure (#18757)
This patch adds a test case to replication_test.py test_replication_with_broker_failure which validates the scenario when we have failures of a combined mode broker/controller.

Reviewers: David Arthur <mumrah@gmail.com>
2025-02-18 12:58:58 -05:00
Parker Chang eb46d5c40d
MINOR: Fix the missing and updated licenses (#18950)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 01:30:43 +08:00
PoAn Yang 1132f08c57
KAFKA-18773 Migrate the log4j1 config to log4j 2 for native image and README (#18872)
- update reflection-config.json and resource-config.json to include log4j2 and jackson
- remove unused jackson scala library
- fix the incorrect path of log4j2.yaml
- adopt workaround (--standalone) to make this PR work and it will be fixed by KAFKA-18737)

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 00:48:46 +08:00
TaiJuWu 934b0159bb
KAFKA-18089: Upgrade Caffeine lib to 3.1.8 (#18004)
- Fixed the RemoteIndexCacheTest that fails with caffeine > 3.1.1

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2025-02-18 21:51:38 +05:30
TaiJuWu 4c8d96c0f0
KAFKA-18767: Add client side config check for shareConsumer (#18850)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-18 15:57:56 +00:00
Parker Chang ed366e6b89
MINOR: Align assertFutureThrows method signature with JUnit conventions (#18825)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-18 15:56:42 +00:00
Mickael Maison 0a2fab9310
KAFKA-14484: Decouple UnifiedLog and RemoteLogManager (#18460)
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2025-02-18 15:10:31 +01:00
Andrew Schofield 6c14f64245
MINOR: Rename NoOpShareStatePersister for consistency (#18933)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-18 14:07:59 +00:00
Chirag Wadhwa 63229a768c
KAFKA-16718 [1/n]: Added DeleteShareGroupOffsets request and response schema (#18927)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-18 14:06:24 +00:00
Bruno Cadonna d6b6952d48
KAFKA-18736: Add Streams group heartbeat request manager (1/N) (#18870)
This commit adds the Streams group heartbeat request manager
to the async consumer. The Streams group heartbeat request
manager is responsible to send heartbeat requests and to
process their responses.

This commit implements:
- sending of full heartbeat request (independent of any state)
- processing successful response

Reviewers: Bill Bejeck <bill@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2025-02-18 13:45:01 +01:00
David Jacot 657154dfb8
MINOR: Update LICENSE-binary (#18943)
Before the patch:
```
% python3 ./committer-tools/verify_license.py

...

All libs from ./libs are present in the LICENSE file.

The following entries are in the LICENSE file but not present in ./libs. These should be removed from the LICENSE-binary file:
 - audience-annotations-0.12.0
 - jackson-jaxrs-base-2.16.2
 - jackson-jaxrs-json-provider-2.16.2
 - jackson-module-jaxb-annotations-2.16.2
 - jakarta.inject-2.6.1
 - javax.servlet-api-3.1.0
 - jetty-continuation-9.4.56.v20240826
 - jetty-servlet-9.4.56.v20240826
 - jetty-servlets-9.4.56.v20240826
 - jetty-util-ajax-9.4.56.v20240826
 - jsr305-3.0.2
 - log4j-core-test-2.24.1
```

After the patch:
```
% python3 ./committer-tools/verify_license.py

...

All libs from ./libs are present in the LICENSE file.

No extra dependencies in the LICENSE file.
```

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-02-18 13:24:03 +01:00
David Jacot 5413063441
MINOR: Add verify_license tool (#18931)
This patch adds the verify_license.py tool. It compares the libraries shipped within the tarball to the LICENSE file, and vice versa, to ensure that they are aligned. It also slightly update the format of the LICENSE file to make it easier to parse it.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
2025-02-18 12:07:37 +01:00
Andrew Schofield 385b7ad355
MINOR: Align share group admin authz with consumer group (#18936)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-02-18 09:12:07 +00:00
Luke Chen 1f47a78a10
MINOR: add docs for "org.apache.kafka.sasl.oauthbearer.allowed.urls" (#18938)
add docs for "org.apache.kafka.sasl.oauthbearer.allowed.urls"

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2025-02-18 16:47:43 +08:00
xijiu 1dcdbf78bb
KAFKA-18798 The replica placement policy used by ReassignPartitionsCommand is not aligned with kraft controller (#18914)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-18 16:02:00 +08:00
Kamal Chandraprakash da3643c6b4
KAFKA-18787: RemoteIndexCache fails to delete invalid files on init (#18888)
The stale/invalid files that ends-with ".deleted" and ".tmp" should be cleaned when the broker gets restarted.

- fix the remote-index-cache test to use the logDir instead of topicDir
- fix the flaky test

Reviewers: Luke Chen <showuon@gmail.com>
2025-02-18 12:56:03 +05:30
Sean Quah 1c9190d6b1
KAFKA-18807; Fix thread idle ratio metric (#18934)
When group.coordinator.threads is greater than 1, we lose track of thread idle time because of integer arithmetic. Use doubles instead.

Reviewers: David Jacot <djacot@confluent.io>
2025-02-18 08:11:38 +01:00
David Jacot 2ecc16b987
MINOR: Remove dropwizard metrics from dependencies.gradle (#18932)
This patch removes dropwizard metrics in the dependency list as it is not used any more. It was introduced in 4f5b4c868e because it was required by Zookeeper. Zookeeper is no longer there so we can remove it too.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-02-18 08:10:06 +01:00
Matthias J. Sax 87f797811b
HOTFIX: StoreChangelogReader should require stable consumer group (#18901)
Fixing regression bug, introduced by beac86f049

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bruno Cadonna <bruno@confluent.io>
2025-02-17 12:53:13 -08:00
Kaushik Raina 35420eb11b
KAFKA-18684: Add base exception classes (#18871)
Introduced two new exception classes to the Kafka error handling framework:

ApplicationRecoverableException: This exception signals that the error is recoverable, but the producer needs to be restarted. It helps in scenarios where recovery actions (like re-balancing or restoring from checkpoints) are needed.

RefreshRetriableException: This exception occurs when metadata is outdated or invalid and needs to be refreshed before retrying the request. It helps handle retries that depend on updated metadata.

Both classes are abstract and in upcoming PRs they will be extended by relevant classes as mentioned in KIP-1050:Exception Table.

Reviewers: Justine Olshan <jolshan@confluent.io>, Sanskar Jhajharia <jhajharia.sanskar@gmail.com>
2025-02-17 12:11:51 -08:00
Apoorv Mittal 06ce3e890b
KAFKA-18733: Updating share group record acks metric (2/N) (#18924)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-17 18:12:58 +00:00
Jimmy Wang 98a7ce5caa
KAFKA-18801 Remove ClusterGenerator and revise ClusterTemplate javadoc (#18907)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2025-02-17 10:39:21 -05:00
Lucas Brutschy d0c65a1fd2
KAFKA-18730: Add replaying streams group state from offset topic (#18809)
Adds streams group to the GroupMetadataManager, and implements loading
the records from the offset topic into state. The state also contains
two timers (rebalance timeout and session timeout) that are started
after the group coordinator has been loaded.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-02-17 16:13:21 +01:00
PoAn Yang 2b6e868538
KAFKA-18784 Fix ConsumerWithLegacyMessageFormatIntegrationTest (#18889)
In PR #18267, we removed old message format for cases in ConsumerWithLegacyMessageFormatIntegrationTest. Although test cases can pass, they don't fulfill original purpose. We can't send old message format since 4.0, so I change cases to append old records by ReplicaManager directly.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 20:43:29 +08:00