Commit Graph

15956 Commits

Author SHA1 Message Date
Abhinav Dixit 14ed1162a5
KAFKA-19310: (MINOR) Missing mocks for DelayedShareFetchTest tests related to Memory Records slicing (#19823)
### About
Added test memory records to avoid the silent exception thrown during
slicing.

### Testing
Ran the tests of `DelayedShareFetchTest` to make sure that there is no
silent exception in any test.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-28 14:01:00 +01:00
PoAn Yang d6ee83a893
KAFKA-17747: [4/N] Replace subscription metadata with metadata hash in consumer group (#19761)
* Add `topicHashCache` to `GroupMetadataManager`.
* Remove topic hash from cache if related topic image is updated.
* Ignore topic hash 0 when calculating group metadata hash.
* Add `metadataHash` to `ModernGroup`.
* Replace subscription metadata with metadata hash.
* If there is data in `ConsumerGroupPartitionMetadataValue`, set a flag
in group to add tombstone record in next group heartbeat.

Reviewers: David Jacot <djacot@confluent.io>
2025-05-28 05:56:46 -07:00
S.Y. Wang 543fb6c848
KAFKA-19336 Upgrade Jackson to 2.19.0 (#19835)
`JsonNode.fields()` method has been deprecated by 
- https://github.com/FasterXML/jackson-databind/issues/4863
- https://github.com/FasterXML/jackson-databind/pull/4871

So modified accordingly.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-28 20:53:43 +08:00
Andrew Schofield 5a607db6ea
MINOR: Improve share coordinator record schemas (#19830)
CI / build (push) Waiting to run Details
This PR makes some very small improvements to the record schemas for the
share coordinator.

* It removes the health warnings about incompatible changes. All changes
are compatible now.
* It marks the fields in the values as version 0+, in common with all
other record schemas in Kafka.
Many were already 0+, so this just corrects the outliers.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan
<smahajan@confluent.io>
2025-05-28 10:24:17 +01:00
Alieh Saeedi a3d5ca07f8
MINOR: Change `Streams group` to `streams group` (#19813)
As of https://github.com/apache/kafka/pull/19758#discussion_r2097734386,
the capitalization across all messages are aligned.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-05-28 09:56:29 +02:00
Andrew Schofield 591a24eaaf
MINOR: Align tool output for deletion of groups (#19829)
This PR makes some trivial improvements to the output formatting of the
tools that are used to delete consumer groups and share groups.

* In general, group IDs are enclosed in single quotes. The PR uses the
same conventions for lists of groups (comma-separated with each group ID
in single quotes).
* The PR also uses the same grammar for both kinds of groups.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-28 07:31:03 +01:00
Ming-Yen Chung fafc140c83
MINOR: Migrate usage of deprecated Endpoint#listenerName() to Endpoint#listener() (#19825)
[KIP-1143](https://cwiki.apache.org/confluence/x/LwqWF) replaced
`Optional<String> listenerName()` with `String listener()`.

This patch migrates the usage from the deprecated method to the new one.

Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>, Yung <yungyung7654321@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>
2025-05-28 11:12:43 +08:00
Sanskar Jhajharia 0fd7ed5ec3
MINOR: Cleanup Core Module- Scala Modules (2/n) (#19510)
Now that Kafka Brokers support Java 17, this PR makes some changes in
core module. The changes in this PR are limited to only the Scala files
in the Core module's tests. The unit tests module is still pending. It
shall follow next. The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

To be clear, the directories being targeted in this PR are:
KafkaApisTest.scala

Reviewers: PoAn Yang <payang@apache.org>, Ming-Yen Chung
<mingyen066@gmail.com>, Yung <yungyung7654321@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>
2025-05-28 10:27:06 +08:00
Ritika Reddy 17226faa80
MINOR: Fix initProducerId throttle time to non-zero (#19801)
CI / build (push) Waiting to run Details
There was a small error in setting the throttle time in the
InitProducerId response in this PR

[53e95ffcdb (diff-1f7f71b4fbeaf65123e65cf2e4d9c2ae8153820869eb5e88279df979130929d2R69)](url),
which was supposed to be mostly mechanical changes.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-05-27 14:57:23 -07:00
David Jacot 25031373da
KAFKA-19154; Offset Fetch API should return INVALID_OFFSET if requested topic id does not match persisted one (#19744)
CI / build (push) Waiting to run Details
This patch updates the OffsetFetch API to ensure that a committed offset
is returned iff the requested topic id matches the persisted one; the
invalid offset is returned otherwise.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-05-27 16:15:06 +02:00
Abhinav Dixit d9233d2f16
MINOR: Remove flakiness caused when producing aborted transaction in ShareConsumerTest (#19824)
This PR attempts to removed the flakiness in
`testAlterReadCommittedToReadUncommittedIsolationLevelWithReleaseAck`
and `testAlterReadCommittedToReadUncommittedIsolationLevelWithRejectAck`.
This flakiness could potentially be caused because we were not ensuring
that the aborted transaction record produce happened. In this PR, I have
added a blocking call to make sure the produce future completes before
we abort the transaction.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-27 12:52:07 +01:00
Ming-Yen Chung 0013d78298
KAFKA-19330 Change MockSerializer/Deserializer to use String serializer instead of byte[] (#19812)
While rewriting `EndToEndClusterIdTest` in Java (#19741 ), I found that
the test uses `MockInterceptor` and `MockSerializer` together. However,
`MockSerializer` was using a `byte[]` serializer, while
`MockInterceptor` expected a `String` serializer, leading to a
`ClassCastException`.

I chose to update `MockSerializer` to use `String`, as it is used less
frequently than the `MockInterceptor`. Using `String` also simplifies
the code  by avoiding the need to write expressions like
`"value".getBytes`.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-27 18:20:34 +08:00
Federico Valeri ad849f1f27
MINOR: Upgrade to Gradle 8.14.1 (#19780)
This patch upgrades to Gradle 8.14.1.  Note that we are switching to
using the recommended "-bin" package.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>
2025-05-27 18:04:58 +08:00
Dongnuo Lyu fcb722dc88
KAFKA-18687: Setting the subscriptionMetadata during conversion to consumer group (#19790)
CI / build (push) Waiting to run Details
When a consumer protocol static member replaces an existing member in a
classic group, it's not necessary to recompute the assignment. However,
it happens anyway.

In

[ConsumerGroup.fromClassicGroup](0ff4dafb7d/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java (L1140)),
we don't set the group's subscriptionMetadata.  Later in the consumer
group heartbeat, we [call

updateSubscriptionMetadata](0ff4dafb7d/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java (L1748)),
which [notices that the group's subscriptionMetadata needs an

update](0ff4dafb7d/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java (L2757))
and bumps the epoch. Since the epoch is bumped, we [recompute the

assignment](0ff4dafb7d/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java (L1766)).

As a fix, this patch sets the subscriptionMetadata in
ConsumerGroup.fromClassicGroup.

Reviewers: Sean Quah <squah@confluent.io>, David Jacot <djacot@confluent.io>
2025-05-27 02:25:57 -07:00
Ji-Seung Ryu 77aff85b3e
KAFKA-19268 Missing mocks for SharePartitionManagerTest tests (#19786)
CI / build (push) Waiting to run Details
Added missing mocks for SharePartitionManagerTests.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-05-26 22:17:44 +01:00
YuChia Ma 6e380fbbcc
KAFKA-19322 Remove the DelayedOperation constructor that accepts an external lock (#19798)
CI / build (push) Waiting to run Details
Remove the DelayedOperation constructor that accepts an external lock.

According to this [PR](https://github.com/apache/kafka/pull/19759).

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-27 01:05:41 +08:00
Nick Guo 0600abdde3
KAFKA-19300 AsyncConsumer#unsubscribe always timeout due to GroupAuthorizationException (#19779)
I verified the behavior by rewriting the
`GroupAuthorizerIntegrationTest` in Java in this PR:
https://github.com/apache/kafka/pull/19685  The state is now correct.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-27 00:52:56 +08:00
TengYao Chi 48a52701b9
KAFKA-18424: Consider splitting PlaintextAdminIntegrationTest#testConsumerGroups (#19093)
CI / build (push) Waiting to run Details
JIRA: KAFKA-18424
`PlaintextAdminIntegrationTest#testConsumerGroups` test too many things.
We should split it into smaller units.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-26 12:10:49 +08:00
Sanskar Jhajharia 2fe447a8a3
MINOR: Cleanups in Test Common Module (#19775)
CI / build (push) Waiting to run Details
Now that Kafka Brokers support Java 17, this PR makes some changes in
test-common module. The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-26 03:07:30 +08:00
Jhen-Yung Hsu 651f86b77e
MINOR: Remove unused mkMapOfPartitionRacks method (#19797)
The mkMapOfPartitionRacks in ServerSideAssignorBenchmark.java was
introduced in 8013657f5d,  and the one in
GroupCoordinatorRecordHelpersTest.java was introduced in
3709901c9e.

Both have not been used since bb97d63d41.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-26 02:54:17 +08:00
Ken Huang bcda92b5b9
KAFKA-19080 The constraint on segment.ms is not enforced at topic level (#19371)
CI / build (push) Waiting to run Details
The main issue was that we forgot to set
`TopicConfig.SEGMENT_BYTES_CONFIG` to at least `1024 * 1024`, which
caused problems in tests with small segment sizes.

To address this, we introduced a new internal config:
`LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG`, allowing us to set smaller
segment bytes specifically for testing purposes.

We also updated the logic so that if a user configures the topic-level
segment bytes without explicitly setting the internal config, the
internal value will no longer be returned to the user.

In addition, we removed
`MetadataLogConfig#METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG` and added
three new internal configurations:
- `INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_DELETE_DELAY_MILLIS_CONFIG`

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-25 20:57:22 +08:00
Hong-Yi Chen 69a457d8a5
KAFKA-19034 [1/N] Rewrite RemoteTopicCrudTest by ClusterTest and move it to storage module (#19681)
CI / build (push) Waiting to run Details
This PR rewrites `RemoteTopicCrudTest` in Java using the `@ClusterTest`
framework and moves it to the `storage` module.

**Note:** Two test cases have not yet been migrated

- `testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic`
-

`testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled`

These tests rely on modifying broker configs during the test lifecycle,
which `ClusterTest` currently does not support. They will be migrated in
a follow-up PR after
[#16808](https://github.com/apache/kafka/pull/16808) is merged, which
introduces support for config updates in `ClusterTest`.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-25 14:50:16 +08:00
Apoorv Mittal adb76779ed
KAFKA-19312 Avoiding concurrent execution of onComplete and tryComplete (#19759)
The `onComplete` method in DelayedOperation is guaranteed to run only
once, through `forceComplete`, invoked either by `tryComplete` or when
operation is expired (`run` method). The invocation of  `tryComplete` is
done by attaining `lock` so no concurrent execution of  `tryComplete`
happens for same delayed operation. However, there can be  concurrent
execution of `tryComplete` and `onComplete` as the  `expiration` thread
can trigger a separte run of `onComplete` while  `tryComplete` is still
executing. This behaviour is not ideal as there  are parallel runs where
1 threads method execution is wasteful i.e. if  `onComplete` is already
invoked by another thread then execution of  `tryComplete` is not
required.

I ran some tests and performance is same.

### After the chages:

```
--num 10000 --rate 100 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 7
# interval samples: rate = 100.068948, min = 0, max = 129
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
101196	99.809364	99.806376	3240	0 2	0 8
```

```
--num 10000 --rate 1000 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 9
# interval samples: rate = 999.371395, min = 0, max = 14
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
11104	989.902990	989.805008	1349	0 2	0 7
```

### Before changes:

```
--num 10000 --rate 100 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 9
# interval samples: rate = 100.020304, min = 0, max = 130
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
102366	98.657274	98.652408	3444	0 2	0 8

--num 10000 --rate 1000 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 8
# interval samples: rate = 997.134236, min = 0, max = 14
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
11218	978.665101	978.665101	1624	0 2	0 7

Reviewers: Jun Rao <junrao@gmail.com>, Andrew Schofield
 <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-25 14:36:43 +08:00
Yu-Syuan Jheng 1407b12e2f
KAFKA-19313 Replace LogOffsetMetadata#UNIFIED_LOG_UNKNOWN_OFFSET by UnifiedLog.UNKNOWN_OFFSET (#19767)
CI / build (push) Waiting to run Details
Replaces the UNIFIED_LOG_UNKNOWN_OFFSET constant in LogOffsetMetadata
with UnifiedLog.UNKNOWN_OFFSET.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, YuChia Ma <minecraftmiku831@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
2025-05-24 23:33:26 +08:00
Alyssa Huang 97db06689b
KAFKA-18345; Wait the entire election timeout on election loss (#19747)
CI / build (push) Waiting to run Details
Replaces exponential backoff for candidate state after losing election
with waiting rest of election timeout. There is no need to have an
exponential backoff when the election timeout already provides a natural
throttle and it is randomized.

Reviewers: José Armando García Sancio <jsancio@apache.org>, TaiJuWu
 <tjwu1217@gmail.com>
2025-05-23 12:35:49 -04:00
Sushant Mahajan af4d048da6
MINOR: Bugfix in GroupMetadataManager.testShareGroupInitializeSuccess. (#19795)
Currently, we were asserting on records containing set using
`assertEquals` which can fail intermittently. To fix the assertion has
been replaced by `assertRecordEquals`.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-23 23:54:38 +08:00
Ken Huang bbe27e65a3
MINOR: Fix the CLIENT_QUOTA_CALLBACK_CLASS_CONFIG document (#18713)
See the discussion:

7fa0dfd72d (r1929621640)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 23:49:26 +08:00
Yunchi Pang 7da9457b36
KAFKA-19260 Move LoggingController to server module (#19687)
CI / build (push) Waiting to run Details
Move `LoggingController` to server module and rewrite it in java.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-23 23:39:33 +08:00
Ming-Yen Chung e107e69a51
HOTFIX: Fix GroupMetadataManager#completeAlterShareGroupOffsets to use InitMapValue in addInitializingTopicsRecords (#19792)
https://github.com/apache/kafka/pull/19781/files#diff-00f0f81cf13e66781777d94f7d2e68a581663385c37e98792507f2294c91bb09L2746-R2745
changes the `addInitializingTopicsRecords` signature while
https://github.com/apache/kafka/pull/18929/files#r2104172356 didn't make
adjustment accordingly.

Fix GroupMetadataManager#completeAlterShareGroupOffsets to use
`InitMapValue` in `initializingTopics` so  that
`addInitializingTopicsRecords` can accept it.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Sushant Mahajan <sushant.mahajan88@gmail.com>, Kuan-Po Tseng <brandboat@gmail.com>, TengYao Chi <frankvicky@apache.org>, Shivsundar R <shr@confluent.io>, PoAn Yang <payang@apache.org>, Nick Guo <lansg0504@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>
2025-05-23 13:07:37 +01:00
jimmy b44bfca408
KAFKA-16717 [2/N]: Add AdminClient.alterShareGroupOffsets (#18929)
[KAFKA-16720](https://issues.apache.org/jira/browse/KAFKA-16720) aims to
finish the AlterShareGroupOffsets RPC.

Reviewers: Andrew Schofield <aschofield@confluent.io>

---------

Co-authored-by: jimmy <wangzhiwang@qq.com>
2025-05-23 09:05:48 +01:00
Jhen-Yung Hsu 999afbbbf1
KAFKA-19069 DumpLogSegments does not dump the LEADER_CHANGE record (#19729)
## Description:
Add case for dumping LEADER_CHANGE record in DumpLogSegments.

- Extract the part of printing ControlRecord as a method.
- Add the unit test for printing the control record in dumpLog.

## Verification

### Before:
```
$ bin/kafka-dump-log.sh --cluster-metadata-decoder --files
try_KAFKA-19069/00000000000000000000.log
Dumping try_KAFKA-19069/00000000000000000000.log
Log starting offset: 0
baseOffset: 0 lastOffset: 2 count: 3 baseSequence: -1 lastSequence: -1
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1
isTransactional: false isControl: true deleteHorizonMs:
OptionalLong.empty position: 0 CreateTime: 1747299887693 size: 174
magic: 2 compresscodec: none crc: 3714449884 isvalid: true
| offset: 0 CreateTime: 1747299887693 keySize: 4 valueSize: 19 sequence:
-1 headerKeys: [] controlType: LEADER_CHANGE(2)
| offset: 1 CreateTime: 1747299887693 keySize: 4 valueSize: 5 sequence:
-1 headerKeys: [] KRaftVersion {"version":0,"kRaftVersion":1}
| offset: 2 CreateTime: 1747299887693 keySize: 4 valueSize: 55 sequence:
-1 headerKeys: [] KRaftVoters
{"version":0,"voters":[{"voterId":1,"voterDirectoryId":"tpx8pd1IQDqJwMfGzXj6GA","endpoints":[{"name":"CONTROLLER","host":"localhost","port":9093}],"kRaftVersionFeature":{"minSupportedVersion":0,"maxSupportedVersion":1}}]}
```

### After:
```
$ bin/kafka-dump-log.sh --cluster-metadata-decoder --files
try_KAFKA-19069/00000000000000000000.log
Dumping try_KAFKA-19069/00000000000000000000.log
Log starting offset: 0
baseOffset: 0 lastOffset: 2 count: 3 baseSequence: -1 lastSequence: -1
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1
isTransactional: false isControl: true deleteHorizonMs:
OptionalLong.empty position: 0 CreateTime: 1747299887693 size: 174
magic: 2 compresscodec: none crc: 3714449884 isvalid: true
| offset: 0 CreateTime: 1747299887693 keySize: 4 valueSize: 19 sequence:
-1 headerKeys: [] leaderChange:
{"version":0,"leaderId":1,"voters":[{"voterId":1}],"grantingVoters":[{"voterId":1}]}
| offset: 1 CreateTime: 1747299887693 keySize: 4 valueSize: 5 sequence:
-1 headerKeys: [] KRaftVersion {"version":0,"kRaftVersion":1}
| offset: 2 CreateTime: 1747299887693 keySize: 4 valueSize: 55 sequence:
-1 headerKeys: [] KRaftVoters
{"version":0,"voters":[{"voterId":1,"voterDirectoryId":"tpx8pd1IQDqJwMfGzXj6GA","endpoints":[{"name":"CONTROLLER","host":"localhost","port":9093}],"kRaftVersionFeature":{"minSupportedVersion":0,"maxSupportedVersion":1}}]}
```

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-23 15:58:35 +08:00
Sushant Mahajan c58de75712
KAFKA-19204: Add timestamp to share state metadata init maps [1/N] (#19781)
1.  Currently, the code allows for retrying any initializing topics in
subsequent heartbeats. This can result in duplicate calls to persister
if multiple share consumers join the same group concurrently.
Furthermore, only one of these will succeed as the others will have a
lower state epoch and will be fenced.
2. The existing change was made in
https://github.com/apache/kafka/pull/19603 to allow for retrying
initialization of initializing topics, in case the original caller was
not able to persist the information in the persister due to a dead
broker/timeout.
3. To prevent multiple calls as well as allow for retry we have
supplemented the timelinehashmap holding the
`ShareGroupStatePartitionMetadataInfo` to also hold the timestamp at
which this record gets replayed.
  a. Now when we get multiple consumers for the same group and topic,
only one of them is allowed to make the persister initialize request and
this information is added to the map when it is replayed. Thus solving
issue 1.
  b. To allow for retries, if an initializing topic is found with a
timestamp which is older than 2*offset_write_commit_ms, that topic will
be allowed to be retried. Here too only one consumer would be able to
retry thus resolving issue 2 as well.
4. Tests have been added wherever applicable and existing ones updated.
5. No record schema changes are involved.
6. The `ShareGroupStatePartitionMetadataInfo` and `InitMapValue` records
have been moved to the `ShareGroup` class for better encapsulation.
7. Some logs have been changed from error to info in
`ShareCoordinatorShard` and extra information is logged.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-23 08:56:05 +01:00
Yunchi Pang 5c3d7512de
MINOR: suppress removal warnings for listClientMetricsResources (#19787)
CI / build (push) Waiting to run Details
suppress removal warnings for `listClientMetricsResources`

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 04:07:14 +08:00
Ken Huang aacf3a4b3d
KAFKA-19042 Move PlaintextConsumerPollTest to client-integration-tests module (#19582)
Use Java to rewrite PlaintextConsumerPollTest by new test infra and move
it to client-integration-tests module.

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 04:01:51 +08:00
Jing-Jia Hung bc797b077f
KAFKA-19314 Remove unnecessary code of closing snapshotWriter (#19763)
- Remove redundant close of `snapshotWriter`.
- `snapshotWriter` is already closed by `RaftSnapshotWriter#writer`.

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 03:57:14 +08:00
Ming-Yen Chung 21afc83f8e
HOTFIX: Fix testDeleteConsumerGroupOffsets (#19788)
Fix the change in
https://github.com/apache/kafka/pull/19380/files#r2102980872

Use `LinkedHashMap` instead of `Set` to  preserve the order of elements.

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Sanskar Jhajharia
 <sjhajharia@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 03:45:04 +08:00
Gaurav Narula 8f849f91f1
MINOR: Update Scala to v2.13.16 to support Java 24 (#19782)
Refer

https://docs.scala-lang.org/overviews/jdk-compatibility/overview.html#scala-compatibility-table

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 00:54:51 +08:00
PoAn Yang 30d7c71f09
KAFKA-18904: Add Admin#listConfigResources [2/N] (#19743)
CI / build (push) Waiting to run Details
* Add new functions `listConfigResources(Set<ConfigResource.Type>
configResourceTypes, ListConfigResourcesOptions options)` and
`listConfigResources()` to `Admin` interface.
  * New functions can list all kind of config resource types.
  * If input is a set with a type other than `CLIENT_METRICS` and
request version is 0, return `UnsupportedVersionException`.
* Deprecate functions
`listClientMetricsResources(ListClientMetricsResourcesOptions options)`
and `listClientMetricsResources()`.
* Deprecate classes `ListClientMetricsResourcesResult` and
`ClientMetricsResourceListing`.
* Change `ClientMetricsCommand` to use `listConfigResources`.
* Add integration tests to `PlaintextAdminIntegrationTest.java`.
* Add unit tests to `KafkaAdminClientTest.java`.

Reviewers: Andrew Schofield <aschofield@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-05-22 16:05:35 +01:00
Abhinav Dixit 239dce3e04
KAFKA-19291: Increase the timeout of remote storage share fetch requests in purgatory (#19757)
CI / build (push) Waiting to run Details
### About
Consumer groups have a different timeout `REMOTE_FETCH_MAX_WAIT_MS_PROP`
in delayed fetch purgatory for fetch requests having remote storage
fetch ([code

link](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L1669)).
This is done before the request enters the purgatory, so its easy to
change. At the moment share groups can only have a `waitTimeMs` `of
shareFetch.fetchParams().maxWaitMs` (default value `500ms`) for delayed
share fetch purgatory regardless of whether they are remote
storage/local log fetch.
This PR introduces a way to increase the timeout of remote storage fetch
requests if a remote storage fetch request couldn't complete within
`shareFetch.fetchParams().maxWaitMs`, then we create a timer task which
can be interrupted whenever `pendingFetches` is finished. The change has
been done to avoid the expiration of remote storage share fetch
requests.

### Testing
The code has been tested with the help of unit tests and
`LocalTieredStorage.java`

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-22 07:41:33 +01:00
lorcan e88c10d595
KAFKA-4650: Add unit tests for GraphNode class (#18951)
CI / build (push) Waiting to run Details
Unit tests added for the class GraphNode.

Change applied to GraphNode parentNodes() function to return a copy of
the collection, which is consistent with how the children() collection
is returned.

Reviewers: Bill Bejeck <bbejeck@apache.org>

---------

Co-authored-by: Lorcan <lorcanjames1@gmail.com>
2025-05-21 08:58:24 -04:00
Bolin Lin 388db49191
KAFKA-19287 document all group coordinator metrics (#19749)
add following metrics

1.
kafka.server:type=group-coordinator-metrics,name=event-queue-time-ms-p95
2.

kafka.server:type=group-coordinator-metrics,name=num-partitions,state=[loading|active|failed]
3.

kafka.server:type=group-coordinator-metrics,name=event-processing-time-ms-p95
4.

kafka.server:type=group-coordinator-metrics,name=event-purgatory-time-ms-p95
5.
kafka.server:type=group-coordinator-metrics,name=batch-flush-time-ms-p95

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-21 19:15:07 +08:00
Sanskar Jhajharia c9fcad7e86
MINOR: Cleanup Core Module- Scala Modules (1/n) (#19380)
Now that Kafka Brokers support Java 17, this PR makes some changes in
core module. The changes in this PR are limited to only the Scala files
in the Core module's tests. The unit tests module is still pending. It
shall follow next.  The changes mostly include:

- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

To be clear, the directories being targeted in this PR are:
- core/src/test/scala/kafka
- core/src/test/scala/integration/kafka

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-21 16:10:19 +08:00
Ken Huang 95d31befef
MINOR: update the gradle version for java 24 support (#19760)
Gradle 8.14 starts to support Java 24, so we should update the Gradle
version accordingly.

Reviewers: TengYao Chi <kitingiao@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-21 15:10:56 +08:00
TengYao Chi af11146bc9
MINOR: Add more description to KIP-1148 (#19768)
Given that we will remove the `log.cleaner.enable`, the related config
`log.cleaner.threads` will also have the respective adjustment.  This
patch will add a description to it.

For more details:

https://issues.apache.org/jira/browse/KAFKA-13610?focusedCommentId=17950756&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17950756

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-21 14:58:55 +08:00
Luke Chen f314497b63
MINOR: add 3.9.1 to system tests (#19772)
CI / build (push) Waiting to run Details
Add 3.9.1 to system tests following this release guide:

https://cwiki.apache.org/confluence/display/KAFKA/Release+Process#ReleaseProcess-Afterrelease

Reviewers: TengYao Chi <frankvicky@apache.org>
2025-05-21 14:09:00 +08:00
Sanskar Jhajharia 9711caca41
MINOR: Cleanup Shell Module (#19771)
Now that Kafka supports Java 17, this PR updates the shell module to get
rid of older code. The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
- Arrays.asList() are replaced with List.of()
- Replace switch statement with switch expression

Reviewers: PoAn Yang <payang@apache.org>, Yung
<yungyung7654321@gmail.com>, Ken Huang <s7133700@gmail.com>, TengYao Chi
<frankvicky@apache.org>
2025-05-21 12:47:45 +08:00
Kaushik Raina 896f283e1e
KAFKA-18783 : Extend InvalidConfigurationException related exceptions (#19731)
## Summary
Extend InvalidConfigurationException related exceptions
  - `AuthenticationException`
  - `AuthorizationException`
  - `ClusterAuthorizationException`
  - `TransactionalIdAuthorizationException`
  - `UnsupportedVersionException`
  - `UnsupportedForMessageFormatException`
  - `InvalidRecordException`
  - `InvalidRequiredAcksException`
  - `RecordBatchTooLargeException`
  - `InvalidTopicException`
  - `TopicAuthorizationException`
  - `GroupAuthorizationException`

## Testing
- Added unit tests in `TransactionExceptionHierarchyTest` to verify:
  - All configuration-related exceptions properly extend
`InvalidConfigurationException`

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-05-20 16:31:33 -07:00
Ritika Reddy 37fdbb3575
MINOR:- Edit test in transaction command (#19746)
CI / build (push) Waiting to run Details
The test incorrectly used force-terminate instead of
forceTerminateTransaction

Reviewers: Justine Olshan <jolshan@confuent.io>, Kuan-Po Tseng
 <brandboat@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-05-20 10:23:38 -07:00
Andy Li 38f3b2835d
MINOR: API Responses missing latest version in Kafka protocol guide (#19769)
### Issue: 

API Responses missing latest version in [Kafka protocol
guide](https://kafka.apache.org/protocol.html)

#### For example:

These are missing:

- ApiVersions Response (Version: 4) — Only versions 0–3 are documented,
though version 4 of the request is included.

- DescribeTopicPartitions Response — Not listed at all.

- Fetch Response (Version: 17) — Only versions 4–16 are documented,
though version 17 of the request is included.

#### After the fix:

docs/generated/protocol_messages.html

<img width="1045" alt="image"
src="https://github.com/user-attachments/assets/5ea79ced-aab5-4c47-8e09-9956047c9bf1"
/>

Reviewers: dengziming <dengziming1993@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-21 00:26:51 +08:00
Sanskar Jhajharia 9f293866ab
MINOR: Cleanup Share Coordinator (#19770)
CI / build (push) Waiting to run Details
Now that Kafka Brokers support Java 17, this PR updates the share
coordinator module to get rid of older code. The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
- Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-20 12:33:20 +01:00