Commit Graph

15748 Commits

Author SHA1 Message Date
Bill Bejeck f397cbc14c
KAFKA-19256: Only send IQ metadata on assignment changes (#19691)
CI / build (push) Waiting to run Details
This PR adds changes, so the IQ endpoint information is only sent to
streams group members when there has been a change in the assignments
requiring an update in the streams client host-partition ownership.

The existing IQ integration test passes with no modifications and
updated the `GroupMetadataManagerTest` to cover the new process.

Reviewers: Matthias Sax <mjsax@apache.org>, Lucas Brutschy
 <lucasbru@apache.org>
2025-05-16 16:54:12 -04:00
Jhen-Yung Hsu ced56a320b
MINOR: Move logDirs config out of KafkaConfig (#19579)
CI / build (push) Waiting to run Details
Follow up https://github.com/apache/kafka/pull/19460/files#r2062664349

Reviewers: Ismael Juma <ismael@juma.me.uk>, PoAn Yang
<payang@apache.org>, TaiJuWu <tjwu1217@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-17 00:52:20 +08:00
Chirag Wadhwa 391b604c97
KAFKA-19226: Added test_console_share_consumer.py (#19708)
This PR includes the system test file test_console_share_consumer.py
which tests the functioning of ConsoleShareConsumer

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-16 17:33:46 +01:00
David Jacot 199772adc5
KAFKA-19141; Persist topic id in OffsetCommit record (#19683)
This patch adds the `TopicId` field to the `OffsetCommitValue` record as
a tagged field. It will be later used on the offset fetch path to ensure
that the persisted offset matches the requested one.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Sean Quah
 <squah@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2025-05-16 10:26:36 -04:00
Mickael Maison b0b8f13252
MINOR: Fix broken HTML tag in SESSION_TIMEOUT_MS_DOC (#19740)
The unclosed `<code>` tag breaks the formatting of the docs.

Reviewers: Ken Huang <s7133700@gmail.com>, Lianet Magrans
 <lmagrans@confluent.io>
2025-05-16 10:19:10 -04:00
Federico Valeri 102d35190b
MINOR: Add JmxReporter note to metric.reporters (#19706)
CI / build (push) Waiting to run Details
According to KIP-830, in Kafka 4 users currently relying on JmxReporter
and that are using additional reporters via metric.reporters will have
to include org.apache.kafka.common.metrics.JmxReporter in
metric.reporters.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-05-16 11:58:01 +02:00
Matthias J. Sax 34e426b517
KAFKA-19271: Add internal ConsumerWrapper (#19697)
With KIP-1071 enabled, the main consumer is created differently,  side
stepping `KafkaClientSupplier`.

To allow injection test wrappers, we add an internal ConsumerWrapper,
until we define a new public interface.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-05-16 11:57:37 +02:00
Bolin Lin 21ffc07212
MINOR: migrate MinInSyncReplicasConfigTest to server module (#19727)
Migrate MinInSyncReplicasConfigTest to the server module


Reviewers: PoAn Yang <payang@apache.org>, Yung
 <yungyung7654321@gmail.com>, TengYao Chi <frankvicky@apache.org>, Ken
 Huang <s7133700@gmail.com>
2025-05-16 17:18:28 +08:00
Andrew Schofield 7ae9a26fc2
MINOR: Mark RemoteIndexCacheTest.testConcurrentRemoveReadForCache1 flaky (#19732)
Marking flaky test as a result of 5% failure rate.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-16 09:03:08 +01:00
Sushant Mahajan e70d05ecda
KAFKA-19217: Fix ShareConsumerTest.testComplexConsumer flakiness. (#19734)
* Added alter offset to earliest.
* Reduced produce time to reduce overall test time.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-16 09:00:16 +01:00
Matthias J. Sax 8fc41f9ca2
KAFKA-19171: Kafka Streams crashes with UnsupportedOperationException (#19507)
This PR fixes a regression bug introduced with KAFKA-17203. We need to
pass in mutable collections into `closeTaskClean(...)`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bruno Cadonna <bruno@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2025-05-15 21:37:04 -07:00
Yunchi Pang f55565d321
KAFKA-19267 the min version used by ListOffsetsRequest should be 1 rather than 0 (#19709)
CI / build (push) Waiting to run Details
Updates the min version used by `ListOffsetsRequest` to
`ApiKeys.LIST_OFFSETS.oldestVersion()` rather than hardcoding `1`.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, TengYao Chi <frankvicky@apache.org>, Chia-Ping
 Tsai <chia7712@gmail.com>
2025-05-16 10:33:16 +08:00
Matthias J. Sax bbf48ad677
KAFKA-19208: KStream-GlobalKTable join should not drop left-null-key record (#19580)
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-05-15 18:34:07 -07:00
PoAn Yang c26b09c609
KAFKA-18904: [1/N] Change ListClientMetricsResources API to ListConfigResources (#19493)
* Change `ListClientMetricsResourcesRequest.json` to
`ListConfigResourcesRequest.json`.
* Change `ListClientMetricsResourcesResponse.json` to
`ListConfigResourcesResponse.json`.
* Change `ListClientMetricsResourcesRequest.java` to
`ListConfigResourcesRequest.java`.
* Change `ListClientMetricsResourcesResponse.java` to
`ListConfigResourcesResponsejava`.
* Change `KafkaApis` to handle both `ListClientMetricsResourcesRequest`
v0 and v1 requests.

Reviewers: Andrew Schofield <aschofield@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-05-15 23:39:00 +01:00
Andrew Schofield 567a03dd14
HOTFIX: Parameter mismatch in test for BrokerMetadataPublisher (#19733)
Recent changes to BrokerMetadataPublisher seem to have caused a mismatch
between the BrokerMetadataPublisher and its test.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan
 <smahajan@confluent.io>
2025-05-15 23:20:29 +01:00
Abhinav Dixit 2e0e9b9056
MINOR: Improvements for Persister APIs with respect to share group protocol enablement (#19719)
CI / build (push) Waiting to run Details
### About
Minor improvements to Persister APIs in KafkaApis with respect to share
group protocol enablement

### Testing
The new code added has been tested with the help of unit tests

Reviewers: Sushant Mahajan <sushant.mahajan88@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-15 18:29:16 +01:00
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