Commit Graph

15782 Commits

Author SHA1 Message Date
David Jacot c612cfff29
KAFKA-19274; Group Coordinator Shards are not unloaded when `__consumer_offsets` topic is deleted (#19713)
Group Coordinator Shards are not unloaded when `__consumer_offsets`
topic is deleted. The unloading is scheduled but it is ignored because
the epoch is equal to the current epoch:

```
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1]
Scheduling  unloading of metadata for __consumer_offsets-0 with epoch
OptionalInt[0]
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Scheduling
unloading of metadata for __consumer_offsets-1 with epoch OptionalInt[0]
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Ignored unloading
metadata for __consumer_offsets-0 in epoch OptionalInt[0] since current
epoch is 0.
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Ignored unloading
metadata for __consumer_offsets-1 in epoch OptionalInt[0] since current
epoch is 0.
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
```

This patch fixes the issue by not setting the leader epoch in this case.
The coordinator expects the leader epoch to be incremented when the
resignation code is called. When the topic is deleted, the epoch is not
incremented. Therefore, we must not use it. Note that this is aligned
with deleted partitions are handled too.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, José Armando García Sancio <jsancio@apache.org>
2025-05-15 19:04:38 +02:00
Lianet Magrans 8a577fa5af
MINOR: adding consumer fenced test and log (#19723)
Adding test to specifically force the fencing path due to delayed
rebalance, and validate how the consumer recovers automatically.

Running this test and DEBUG log enabled, allows to see the details of
the fencing flow: consumer getting fenced due to rebalance exceeded,
resetting to epoch 0, rejoining on the next poll with the existing
subscription, and being accepted back in the group (so consumption
resumes)

This is aimed to help understand
[KAFKA-19233](https://issues.apache.org/jira/browse/KAFKA-19233)

Will add another one in separate PR to also involve commits in similar
fencing scenarios.

Reviewers: TengYao Chi <frankvicky@apache.org>
2025-05-15 12:57:11 -04:00
David Arthur d47abb3c27
MINOR: Fix backticks in PR body (#19726)
This fixes the paragraph splitting done in the pr-format.py script. If a
line contains a triple backtick, the script will consider it markdown
and adjust the formatting accordingly.

Reviewers: TengYao Chi <frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>
2025-05-15 12:07:26 -04:00
Sushant Mahajan a26d803f22
MINOR: Remove share group code from group coord onElection. (#19730)
* Previously we had added code to `GroupCoordinatorService.onElection`
to reconcile pending share group initializing topics. This was done to
manage state in case of failovers and broker failures.
* However, we later modified share group heartbeat code to do the retry
to clean up the state and the `onElection` code is now redundant.
* In this PR we are cleaning up this code.

Reviewers: David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-05-15 07:27:25 -07:00
Chirag Wadhwa f9064f8bcd
KAFKA-19231-1: Handle fetch request when share session cache is full (#19701)
According to the recent changes in KIP-932, when the share session cache
is full and a broker receives a Share Fetch request with Initial Share
Session Epoch (0), then the error code `SHARE_SESSION_LIMIT_REACHED` is
returned after a delay of maxWaitMs. This PR implements this logic. In
order to add a delay between subsequent share fetch requests, the timer
is delayed operation purgatory is used. A new `IdleShareFetchTimeTask`
has been added which takes in a CompletableFuture<Void>. Upon the
expiration, this future is completed with null. When the future is
completes, a response is sent back to the client with the error code
`SHARE_SESSION_LIMIT_REACHED`

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-15 14:36:44 +01:00
Sushant Mahajan 847968e530
KAFKA-19281: Add share enable flag to periodic jobs. (#19721)
* We have a few periodic timer tasks in `ShareCoordinatorService` which
run continuously.
* With the recent introduction of share group enabled config at feature
level, we would like these jobs to stop when the aforementioned feature
is disabled.
* In this PR, we have added the functionality to make that possible.
* Additionally the service has been supplemented with addition of a
static share group config supplier.
* New test has been added.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal
 <apoorvmittal10@gmail.com>
2025-05-15 14:05:06 +01:00
YuChia Ma 1c3c361629
KAFKA-19263 update `delete.topic.enable` docs (#19675)
If delete.topic.enable is false on the brokers, deleteTopics will mark
the topics for deletion, but not actually delete them. The futures will
return successfully in this case.

It is not true as the server return exception now.

```java if (!config.deleteTopicEnable) {   if (apiVersion < 3) { throw
new InvalidRequestException("Topic deletion is disabled.")    } else {
throw new TopicDeletionDisabledException()    } } ```

Reviewers: Nick Guo <lansg0504@gmail.com>, PoAn Yang
 <payang@apache.org>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-15 20:03:54 +08:00
Apoorv Mittal 6d45657cde
MINOR: Cleaning up code for share feature listener (#19715)
CI / build (push) Waiting to run Details
The PR is a minor follow up on
https://github.com/apache/kafka/pull/19659.

KafkaApis.scala already have a check which denies new share fetch
related calls if the share group is not supported. Hence no new sessions
shall be created so the requirement to have share group enabled flag in
ShareSessionCache is not needed, unless I am missing something.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-05-15 10:10:54 +01:00
PoAn Yang a1008dc85d
KAFKA-17747: [2/N] Add compute topic and group hash (#19523)
* Add `com.dynatrace.hash4j:hash4j:0.22.0` to dependencies.
* Add `computeTopicHash` to `org.apache.kafka.coordinator.group.Utils`.
  * If topic name is non-existent, return 0.
  * If topic name is existent, use streaming XXH3 to compute topic hash
with magic byte, topic id, topic name, number of partitions, partition
id and sorted racks.
* Add `computeGroupHash` to `org.apache.kafka.coordinator.group.Utils`.
  * If topic map is empty, return 0.
  * If topic map is not empty, use streaming XXH3 to compute group
metadata hash with sorted topic hashes by topic names.
* Add related unit test.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, Sean Quah <squah@confluent.io>, David Jacot <djacot@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-05-15 10:48:45 +02:00
xijiu ecb5b6bd7e
KAFKA-19269 `Unexpected error ..` should not happen when the delete.topic.enable is false (#19698)
Do not print the `"Unexpected error .."` log when the config
`delete.topic.enable` is false

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>, Chia-Ping
 Tsai <chia7712@gmail.com>
2025-05-15 15:17:49 +08:00
Ming-Yen Chung 7d4acedc27
KAFKA-19270: Remove Optional from ClusterInstance#controllerListenerName() return type (#19718)
CI / build (push) Waiting to run Details
In KRaft mode, controllerListenerName must always be specified, so we don't need an `Optional` to wrap it.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-05-15 10:47:47 +08:00
Ritika Reddy 7f02c263a6
KAFKA-19082:[3/4] Add prepare txn method (KIP-939) (#19539)
This patch belongs to the client-side changes required to enable 2PC as
a part of KIP-939.

New method is added to KafkaProducer:  public PreparedTxnState
prepareTransaction()

This would flush all the pending messages and transition the producer
into a mode where only .commitTransaction, .abortTransaction, or
.completeTransaction could be called (calling other methods,  e.g. .send
, in that mode would result in IllegalStateException being thrown).  If
the call is successful (all messages successfully got flushed to all
partitions) the transaction is prepared.  If the 2PC is not enabled, we
return the INVALID_TXN_STATE error.

A new state is added to the TransactionManager called
PREPARING_TRANSACTION. There are two situations where we would move into
this state:
1) When prepareTransaction() is called during an ongoing transaction
with 2PC enabled
2) When initTransaction(true) is called after a client failure
(keepPrepared = true)

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan
 <jolshan@confluent.io>
2025-05-14 15:11:39 -07:00
Kevin Wu 37963256d1
KAFKA-18666: Controller-side monitoring for broker shutdown and startup (#19586)
CI / build (push) Waiting to run Details
This PR introduces the following per-broker metrics:

-`kafka.controller:type=KafkaController,name=BrokerRegistrationState,broker=X`

-`kafka.controller:type=KafkaController,name=TimeSinceLastHeartbeatReceivedMs,broker=X`

and this metric:

`kafka.controller:type=KafkaController,name=ControlledShutdownBrokerCount`

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-05-14 10:59:47 -07:00
PoAn Yang cbfbbe833d
KAFKA-19234: broker should return UNAUTHORIZATION error for non-existing topic in produce request (#19635)
Since topic name is sensitive information, it should return a
TOPIC_AUTHORIZATION_FAILED error for non-existing topic. The Fetch
request also follows this pattern.

Co-authored-by: John Doe <zh2725284321@gmail.com>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu <yungyung7654321@gmail.com>

Reviewers: Jun Rao  <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>, TaiJuWu  <tjwu1217@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Ken Huang  <s7133700@gmail.com>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>
2025-05-14 09:56:09 -07:00
YuChia Ma 05169aa201
MINOR: Add deprecation warning for `log.cleaner.enable` and `log.cleaner.threads` (#19674)
Add a warning message when using log.cleaner.enable to remind users that
this configuration is deprecated. Also, add a warning message for
log.cleaner.threads=0 because in version 5.0, the value must be greater
than zero.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, TengYao Chi <frankvicky@apache.org>, TaiJuWu
<tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-15 00:18:27 +08:00
Xuan-Zhang Gong 15331bbfc4
KAFKA-19273 Ensure the delete policy is configured when the tiered storage is enabled (#19702)
We updated the validation rule for cleanup.policy in remote storage
mode.

If remote log storage is enabled, only cleanup.policy=delete is allowed.
Any other value (e.g. compact, compact,delete) will now result in a
config validation error.

Reviewers: Luke Chen <showuon@gmail.com>, Ken Huang
 <s7133700@gmail.com>, PoAn Yang <payang@apache.org>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-15 00:13:55 +08:00
Uladzislau Blok 053b9e423c
KAFKA-19091 Fix race condition in DelayedFutureTest (#19553)
The root case of flakiness is race condition between worker thread
(thread which is executing the test) and executor-testDelayedFuture
(thread which should execute callback).

It was fixed with TestUtils#waitForCondition to wait until callback will
be done

Test evidence:

Test was running 1000 times with repeated test.

Results: `~/p/kafka (bloku/kafka-19091) [1]> ./gradlew server:test
--tests DelayedFutureTest --fail-fast > res.txt` `~/p/kafka
(bloku/kafka-19091)> grep FAILED res.txt ` `~/p/kafka
(bloku/kafka-19091) [1]>`

res.txt: `> Task :server:test`

`Gradle Test Run :server:test > Gradle Test Executor 14 >
DelayedFutureTest > testDelayedFuture() > repetition 1 of 1000 PASSED`

...

`BUILD SUCCESSFUL in 37m`

Reviewers: Ken Huang <s7133700@gmail.com>, TaiJuWu <tjwu1217@gmail.com>,
 PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-14 23:12:40 +08:00
Dmitry Werner 4f2e3ecad4
MINOR: Remove unused TopicOptionalIdPartition class (#19716)
After merging the

[commit](6f783f8536 (diff-78812e247ffeae6f8c49b1b22506434701b1e1bafe7f92ef8f8708059e292bf0L53)),
the `TopicOptionalIdPartition` class is no longer used anywhere and
should be removed.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-14 20:52:49 +08:00
Apoorv Mittal ec70c44362
KAFKA-19116, KAFKA-19258: Handling share group member change events (#19666)
CI / build (push) Waiting to run Details
The PR adds ShareGroupListener which triggers on group changes, such as
when member leaves or group is empty. Such events are specific to
connection on respective broker. There events help to clean specific
states managed for respective member or group in various caches.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-14 09:52:47 +01:00
Chirag Wadhwa 89cbafeec1
MINOR: changed the default value of metadata_quorum in test methods to quorum.isolated_kraft where quorum.zk was used (#19707)
There were some tests in share_consumer_test, where the default value of
metadata_quorum was quorum.zk. That is change to quorum.isolated_kraft in
this PR

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-14 09:22:43 +01:00
Kirk True 692c7f14d6
KAFKA-19206 ConsumerNetworkThread.cleanup() throws NullPointerException if initializeResources() previously failed (#19569)
Guard against possible `NullPointerExceptions` in
`ConsumerNetworkThread.cleanup()` if
`ConsumerNetworkThread.initializeResources()` previously failed.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-14 14:26:51 +08:00
Kuan-Po Tseng cafe83f928
KAFKA-19275 client-state and thread-state metrics are always "Unavailable" (#19712)
Fix the issue where JMC is unable to correctly display client-state and
thread-state metrics. The root cause is that these two metrics directly
return the `State` class to JMX. If the user has not set up the RMI
server, JMC or other monitoring tools will be unable to interpret the
`State` class. To resolve this, we should return a string representation
of the state instead of the State class in these two metrics.

Reviewers: Luke Chen <showuon@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-14 14:07:32 +08:00
Bolin Lin f01e5aa964
KAFKA-19145 Move LeaderEndPoint to Server module (#19630)
Move LeaderEndPoint to Server module

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>, Chia-Ping
Tsai <chia7712@gmail.com>
2025-05-14 13:47:51 +08:00
José Armando García Sancio a619c6bb95
MINOR; Remove cast for Records' slice (#19661)
CI / build (push) Waiting to run Details
In Java return types are covariant. This means that method override can
override the return type with a subclass.

Reviewers: Jun Rao <junrao@apache.org>, Chia-Ping Tsai
 <chia7712@apache.org>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-14 01:06:38 +01:00
Kirk True c60c83aaba
KAFKA-18847: Refactor OAuth layer to improve reusability 1/N (#19622)
CI / build (push) Waiting to run Details
Rename `AccessTokenRetriever` and `AccessTokenValidator` to
`JwtRetriever` and `JwtValidator`, respectively. Also converting the
factory pattern classes `AccessTokenRetrieverFactory` and
`AccessTokenValidatorFactory` into delegate/wrapper classes
`DefaultJwtRetriever` and `DefaultJwtValidator`, respectively.

These are all internal changes, no configuration, user APIs, RPCs, etc.
were changed.

Reviewers: Manikumar Reddy <manikumar@confluent.io>, Ken Huang
 <s7133700@gmail.com>, Lianet Magrans <lmagrans@confluent.io>

---------

Co-authored-by: Ken Huang <s7133700@gmail.com>
2025-05-13 12:35:20 -04:00
Sean Quah c16c240bd1
KAFKA-18688: Fix uniform homogeneous assignor stability (#19677)
When the number of partitions is not divisible by the number of members,
some members will end up with one more partition than others.
Previously, we required these to be the members at the start of the
iteration order, which meant that partitions could be reassigned even
when the previous assignment was already balanced.

Allow any member to have the extra partition, so that we do not move
partitions around when the previous assignment is already balanced.

Before the PR
```
Benchmark                                 (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt    Score   Error  Units
ServerSideAssignorBenchmark.doAssignment              FULL           RANGE          false          10000                         50         HOMOGENEOUS          1000  avgt    2   26.175          ms/op
ServerSideAssignorBenchmark.doAssignment              FULL           RANGE          false          10000                         50       HETEROGENEOUS          1000  avgt    2  123.955          ms/op
ServerSideAssignorBenchmark.doAssignment       INCREMENTAL           RANGE          false          10000                         50         HOMOGENEOUS          1000  avgt    2   24.408          ms/op
ServerSideAssignorBenchmark.doAssignment       INCREMENTAL           RANGE          false          10000                         50       HETEROGENEOUS          1000  avgt    2  114.873          ms/op
```
After the PR
```
Benchmark                                 (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt    Score   Error  Units
ServerSideAssignorBenchmark.doAssignment              FULL           RANGE          false          10000                         50         HOMOGENEOUS          1000  avgt    2   24.259          ms/op
ServerSideAssignorBenchmark.doAssignment              FULL           RANGE          false          10000                         50       HETEROGENEOUS          1000  avgt    2  118.513          ms/op
ServerSideAssignorBenchmark.doAssignment       INCREMENTAL           RANGE          false          10000                         50         HOMOGENEOUS          1000  avgt    2   24.636          ms/op
ServerSideAssignorBenchmark.doAssignment       INCREMENTAL           RANGE          false          10000                         50       HETEROGENEOUS          1000  avgt    2  115.503          ms/op
```

Reviewers: David Jacot <djacot@confluent.io>
2025-05-13 08:01:14 -07:00
Abhinav Dixit d5ce463ed3
KAFKA-19253: Improve metadata handling for share version using feature listeners (1/N) (#19659)
This PR creates a listener for `SharePartitionManager` to listen to any
changes in `ShareVersion` feature. In case, there is a toggle, we need
to change the attributes in `SharePartitionManager` accordingly.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-13 15:01:03 +01:00
Lianet Magrans 33b9471ac3
MINOR: Improve docs for retries & cleanup (#19595)
Improve docs for retries config, mainly to clarify the expected
behaviour on retries=0  Also remove unused funcs and fix typo.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Ming-Yen Chung
 <mingyen066@gmail.com>, PoAn Yang <payang@apache.org>
2025-05-13 09:36:49 -04:00
Sushant Mahajan bf53561d16
KAFKA-19201: Handle deletion of user topics part of share partitions. (#19559)
* Currently even if a user topic is deleted, its related records are not
deleted with respect to subscribed share groups from the GC and the SC.
* The event of topic delete is propagated from the
BrokerMetadataPublisher to the coordinators via the
`onPartitionsDeleted` method. This PR leverages this method to issue
cleanup calls to the GC and SC respectively.
* To prevent chaining of futures in the GC, we issue async calls to both
GC and SC independently and the methods take care of the respective
cleanups unaware of the other.
* This method is more efficient and transcends issues related to
timeouts/broker restarts resulting in chained future execution issues.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-13 14:22:17 +01:00
David Jacot d154a314e7
KAFKA-14691; Add TopicId to OffsetFetch API (#19515)
This patch extends the OffsetFetch API to support topic ids. From
version 10 of the API, topic ids must be used.

The patch only contains the server side changes and it keeps the version
10 as unstable for now. We will mark the version as stable when the
client side changes are merged in.

Reviewers: TengYao Chi <frankvicky@apache.org>, Lianet Magrans <lmagrans@confluent.io>
2025-05-13 15:10:10 +02:00
Apoorv Mittal b4b73c604b
KAFKA-19245: Updated default locks config for share group (#19705)
Updated default locks config for share groups from 200 to 2000. The
increase in the limit is a result from tests which showed that with
default maxFetchRecords of 500 from client and 200 as locks limit, there
can't be parallel fetch for same partition. Also the tests resulted that
sharing a partition to an index of 3-4 is easily achievable, hence
raised the limit to 4 times of default limit of maxFetchRecords (500).

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-13 13:46:25 +01:00
Sean Quah 1293658cca
KAFKA-19163: Avoid deleting groups with pending transactional offsets (#19496)
When a group has pending transactional offsets but no committed offsets,
we can accidentally delete it while cleaning up expired offsets. Add a
check to avoid this case.

Reviewers: David Jacot <djacot@confluent.io>
2025-05-13 05:10:26 -07:00
Andrew Schofield 86baac103b
MINOR: Improve client error messages for share groups not enabled (#19688)
CI / build (push) Waiting to run Details
As mentioned in
https://github.com/apache/kafka/pull/19378#pullrequestreview-2775598123,
the error messages for a 4.1 share consumer could be clearer for the
different cases for when it cannot successfully join a share group.

This PR uses different error messages for the different cases such as
out-of-date cluster or share groups just not enabled.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-13 10:42:40 +01:00
Bolin Lin 6eafe407bd
MINOR: Fix unchecked type warnings in several test classes (#19679)
* In ConsoleShareConsumerTest, add `@SuppressWarnings("unchecked")`
annotation in method shouldUpgradeDeliveryCount
* In ListConsumerGroupOffsetsHandlerTest, add generic parameters to
HashSet constructors
* In TopicsImageTest, add explicit generic type to Collections.EMPTY_MAP
to fix raw type usage

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-13 14:59:22 +08:00
Alyssa Huang c28f46459a
KAFKA-18345; Prevent livelocked elections (#19658)
CI / build (push) Waiting to run Details
At the retry limit binaryExponentialElectionBackoffMs it becomes
statistically likely that the exponential backoff returned
electionBackoffMaxMs. This is an issue as multiple replicas can get
stuck starting elections at the same cadence.

This change fixes that by added a random jitter to the max election
backoff.

Reviewers: José Armando García Sancio <jsancio@apache.org>, TaiJuWu
 <tjwu1217@gmail.com>, Yung <yungyung7654321@gmail.com>
2025-05-12 16:23:18 -04:00
Jonah Hooper 13fa4537f5
KAFKA-18905; Disable idempotent producer to remove test flakiness (#19644)
As a result of KAFKA-18905 the reassign test will often have test
failures which are unrelated to the actual reassignment of partitions.
This failure is mentioned in KAFKA-9199.

Quote from KAFKA-9199:  "This issue popped up in the reassignment system
test. It ultimately caused the test to fail because the producer was
stuck retrying the duplicate batch repeatedly until ultimately giving
up."

Disabling the idempotent producer circumvents this issue and allows the
reassignment system tests to succeed reliably. The reassignment test
still check that produce batches were not lost.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-05-12 15:41:02 -04:00
Andrew Schofield 7b8633e36f
MINOR: Add deprecation of listConsumerGroups to upgrade.html (#19684)
CI / build (push) Waiting to run Details
As part of KIP-1043, `Admin.listConsumerGroups()` and variants have been
deprecated. This is because there are now 4 types of group and listing
has been consolidated under `Admin.listGroups()`. This PR adds the
deprecation information to the upgrade documentation.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-05-12 18:08:11 +01:00
ChickenchickenLove 62bec20aef
KAFKA-19242: Fix commit bugs caused by race condition during rebalancing. (#19631)
### Motivation
While investigating “events skipped in group

rebalancing” ([spring‑projects/spring‑kafka#3703](https://github.com/spring-projects/spring-kafka/issues/3703))
I discovered a race
condition between
- the main poll/commit thread, and
- the consumer‑coordinator heartbeat thread.

If the main thread enters
`ConsumerCoordinator.sendOffsetCommitRequest()` while the heartbeat
thread is finishing a rebalance (`SyncGroupResponseHandler.handle()`),
the group state transitions in the following order:

```
COMPLETING_REBALANCE  →  (race window)  →  STABLE
```
Because we read the state twice without a lock:
1. `generationIfStable()` returns `null` (state still
`COMPLETING_REBALANCE`),
2. the heartbeat thread flips the state to `STABLE`,
3. the main thread re‑checks with `rebalanceInProgress()` and wrongly
decides that a rebalance is still active,
4. a spurious `CommitFailedException` is returned even though the commit
could succeed.

For more details, please refer to sequence diagram below.  <img
width="1494" alt="image"

src="https://github.com/user-attachments/assets/90f19af5-5e2d-4566-aece-ef764df2d89c"
/>

### Impact
- The exception is semantically wrong: the consumer is in a stable
group, but reports failure.
- Frameworks and applications that rely on the semantics of
`CommitFailedException` and `RetryableCommitException` (for example
`Spring Kafka`) take the wrong code path, which can ultimately skip the
events and break “at‑most‑once” guarantees.

### Fix
We enlarge the synchronized block in
`ConsumerCoordinator.sendOffsetCommitRequest()` so that the consumer
group state is examined atomically with respect to the heartbeat thread:

### Jira
https://issues.apache.org/jira/browse/KAFKA-19242

https: //github.com/spring-projects/spring-kafka/issues/3703

Signed-off-by: chickenchickenlove <ojt90902@naver.com>

Reviewers: David Jacot <david.jacot@gmail.com>
2025-05-12 17:01:29 +02:00
Sean Quah eb3714f022
KAFKA-19160;KAFKA-19164; Improve performance of fetching stable offsets (#19497)
CI / build (push) Waiting to run Details
When fetching stable offsets in the group coordinator, we iterate over
all requested partitions. For each partition, we iterate over the
group's ongoing transactions to check if there is a pending
transactional offset commit for that partition.

This can get slow when there are a large number of partitions and a
large number of pending transactions. Instead, maintain a list of
pending transactions per partition to speed up lookups.

Reviewers: Shaan, Dongnuo Lyu <dlyu@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jaco <djacot@confluent.io>
2025-05-12 00:32:17 -07:00
Matthias J. Sax b66729e231
MINOR: fit HTML markup (#19676)
CI / build (push) Waiting to run Details
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-11 16:20:25 -07:00
Ming-Yen Chung 57ae6d6706
KAFKA-18695 Remove quorum=kraft and kip932 from all integration tests (#19633)
CI / build (push) Waiting to run Details
Currently, the quorum uses kraft by default, so there's no need to
specify it explicitly.

For kip932 and isShareGroupTest, they are no longer used after #19542 .

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-12 01:28:30 +08:00
Kuan-Po Tseng 54fd1361e5
KAFKA-19264 Remove fallback for thread pool sizes in RemoteLogManagerConfig (#19673)
The fallback mechanism for `remote.log.manager.copier.thread.pool.size`
and `remote.log.manager.expiration.thread.pool.size` defaulting to
`remote.log.manager.thread.pool.size` was introduced in KIP-950. This
approach was abandoned in KIP-1030, where default values were changed
from -1 to 10, and a configuration validator enforcing a minimum value
of 1 was added. As a result, this commit removes the fallback mechanism
from `RemoteLogManagerConfig.java` to align with the new defaults and
validation.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-11 23:48:45 +08:00
Yunchi Pang f588fa0643
MINOR: Move TxnTransitMetadata to transaction-coordinator (#19662)
Migrates the `TxnTransitMetadata` class from scala to java, moving it
from to the `transaction-coordinator` module.

Reviewers: PoAn Yang <payang@apache.org>, Nick Guo
 <lansg0504@gmail.com>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-11 22:36:47 +08:00
Nick Guo 707a44a6cb
KAFKA-19068 Eliminate the duplicate type check in creating ControlRecord (#19346)
CI / build (push) Waiting to run Details
jira: https://issues.apache.org/jira/browse/KAFKA-19068

`RecordsIterator#decodeControlRecord` do the type check and then
`ControlRecord` constructor does that again.

we should add a static method to ControlRecord to create `ControlRecord`
with type check, and then `ControlRecord` constructor should be changed
to private to ensure all instance is created by the static method.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-11 00:07:00 +08:00
PoAn Yang 61cb33f347
KAFKA-19109 Don't print null in kafka-metadata-quorum describe status (#19543)
If directory id is `Uuid.ZERO_UUID`, the command don't print the result.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-10 23:48:05 +08:00
xijiu 3696c49788
KAFKA-19220 Add tests to ensure the internal configs don't return by public APIs by default (#19650)
Add tests to check whether the results returned by the API
`createTopics` and `describeConfigs` contain internal configurations.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>, TaiJuWu
 <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-10 23:13:58 +08:00
Hong-Yi Chen a7eae28a67
MINOR: Replaced internal KafkaConfig field in TransactionLogConfig (#19482)
Retaining a reference to ```AbstractConfig``` introduced coupling and
potential inconsistencies with dynamic config updates. This change
simplifies ```TransactionLogConfig``` into a POJO by removing the
internal ```AbstractConfig``` field, and aligns with feedback from
#19439

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
 <frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-10 23:06:18 +08:00
Stanislav Kozlovski 0bc8d0c962
MINOR: Add documentation about KIP-405 remote reads serving just one partition per FetchRequest (#19336)
[As discussed in the mailing
list](https://lists.apache.org/thread/m03mpkm93737kk6d1nd6fbv9wdgsrhv9),
the broker only fetches remote data for ONE partition in a given
FetchRequest. In other words, if a consumer sends a FetchRequest
requesting 50 topic-partitions, and each partition's requested offset is
not stored locally - the broker will fetch and respond with just one
partition's worth of data from the remote store, and the rest will be
empty.

Given our defaults for total fetch response is 50 MiB and per partition
is 1 MiB, this can limit throughput. This patch documents the behavior
in 3 configs - `fetch.max.bytes`, `max.partition.fetch.bytes` and
`remote.fetch.max.wait.ms`

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash
 <kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
2025-05-10 16:48:55 +05:30
Alyssa Huang 042be5b9ac
MINOR: Fix some Request toString methods (#19655)
CI / build (push) Waiting to run Details
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-05-09 23:42:34 -07:00
Chia-Ping Tsai 064afe2c65
MINOR: add xijiu from asf.yaml in order to resend invitation (#19663)
CI / build (push) Waiting to run Details
@xijiu's invitation is timeout, so we have to remove the name, then
re-add it in a new commit.

see https://issues.apache.org/jira/browse/INFRA-26796

Reviewers: PoAn Yang <payang@apache.org>, xijiu <422766572@qq.com>,
 Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ken Huang <s7133700@gmail.com>,
 TengYao Chi <frankvicky@apache.org>
2025-05-10 05:42:54 +08:00