Commit Graph

15847 Commits

Author SHA1 Message Date
Kaushik Raina 8c71ab03b5
KAFKA-19176: Update Transactional producer to translate retriable into abortable exceptions (#19522)
CI / build (push) Waiting to run Details
### Problem
- Currently, when a transactional producer encounters retriable errors
(like `COORDINATOR_LOAD_IN_PROGRESS`) and exhausts all retries, finally
returns retriable error to Application Layer.
- Application reties can cause duplicate records. As a fix we are
transitioning all retriable errors  as Abortable Error in transaction
producer path.

- Additionally added InvalidTxnStateException as part of
https://issues.apache.org/jira/browse/KAFKA-19177

### Solution
- Modified the TransactionManager to automatically transition retriable
errors to abortable errors after all retries are exhausted. This ensures
that applications can abort transaction when they encounter
`TransactionAbortableException`

- `RefreshRetriableException` like `CoordinatorNotAvailableException`
will be refreshed internally

[[code](6c26595ce3/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java (L1702-L1705))]
till reties are expired, then it will be treated as retriable errors and
translated to `TransactionAbortableException`

- Similarly for InvalidTxnStateException

### Testing
Added test `testSenderShouldTransitionToAbortableAfterRetriesExhausted`
to verify in sender thread:
- Retriable errors are properly converted to abortable state after
retries
- Transaction state transitions correctly and subsequent operations fail
appropriately with TransactionAbortableException

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-06-03 10:21:22 -07:00
PoAn Yang 2977cb17d0
KAFKA-17747: [6/N] Replace subscription metadata with metadata hash in share group (#19796)
* Use metadata hash to replace subscription metadata.
* Remove `ShareGroupPartitionMetadataKey` and
`ShareGroupPartitionMetadataValue`.
* Use `subscriptionTopicNames` and `metadataImage` to replace
`subscriptionMetadata` in `subscribedTopicsChangeMap` function.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot
<djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-06-03 16:30:39 +01:00
Kaushik Raina 82ea9d0fce
MINOR : Handle error for client telemetry push (#19881)
Update catch to handle compression errors

Before :

![image](https://github.com/user-attachments/assets/c5ca121e-ba0c-4664-91f1-20b54abf67cc)

After
```
Sent message: KR Message 376
[kafka-producer-network-thread | kr-kafka-producer] INFO
org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter -
KR: Failed to compress telemetry payload for compression: zstd, sending
uncompressed data
Sent message: KR Message 377
```

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-03 14:29:44 +01:00
Andrew Schofield 016a4a6c4c
KAFKA-19353: Upgrade note and initial docs for KIP-932 (#19863)
CI / build (push) Waiting to run Details
This is the initial documentation for KIP-932 preview in AK 4.1. The aim
is to get very minimal docs in before the cutoff. Longer term, more
comprehensive documentation will be provided for AK 4.2.

The PR includes:
* Generation of group-level configuration documentation
* Add link to KafkaShareConsumer to API docs
* Add a summary of share group rational to design docs
* Add basic operations information for share groups to ops docs
* Add upgrade note describing arrival of KIP-932 preview in 4.1

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>

---------

Co-authored-by: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-06-03 13:23:11 +01:00
PoAn Yang 425f028556
KAFKA-17747: [5/N] Replace subscription metadata with metadata hash in stream group (#19802)
* Use metadata hash to replace subscription metadata.
* Remove `StreamsGroupPartitionMetadataKey` and
`StreamsGroupPartitionMetadataValue`.
* Check whether `configuredTopology` is empty. If it's, call
`InternalTopicManager.configureTopics` and set the result to the group.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-06-03 13:21:34 +02:00
Sushant Mahajan df93571f50
KAFKA-19338: Error on read/write of uninitialized share part. (#19861)
- Currently, read and write share state requests were allowed on
uninitialized share partitions (share partitions on which
initializeState has NOT been called). This should not be the case.
- This PR addresses the concern by adding error checks on read and
write. Other requests are allowed (initialize, readSummary, alter).
- Refactored `ShareCoordinatorShardTest` to reduce redundancy and added
some new tests.
- Some request/response classes have also been reformatted.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-03 11:26:38 +01:00
Chirag Wadhwa c5a78b0186
KAFKA-19358: Updated share_consumer_test.py tests to use set_group_offset_reset_strategy (#19878)
CI / build (push) Waiting to run Details
According to the current code in AK, the offset reset strategy for share
groups was set using the flag `--offset-reset-strategy` in the
share_consumer_test.py tests, but that would mean that the admin client
call would be sent out by all members in the share group. This PR
changes that by introducing `set_group_offset_reset_strategy` method in
kafka.py, which runs the kafka-configs.sh script in one of the existing
docker containers, thereby changing the config only once.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-02 21:13:22 +01:00
Nick Guo 32903a1873
KAFKA-19349 Move CreateTopicsRequestWithPolicyTest to clients-integration-tests (#19849)
Move CreateTopicsRequestWithPolicyTest to clients-integration-tests.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-03 02:52:07 +08:00
PoAn Yang 78ea8782ae
KAFKA-18904: [4/N] Add ListClientMetricsResources metric if request is v0 ListConfigResources (#19877)
Before 4.1, the api key 74 is `ListClientMetricsResources`. After 4.1,
it's `ListConfigResources`. If users sent a v0 ListConfigResources to
broker, the metric doesn't record request with
`ListClientMetricsResources`. This PR is to add
`ListClientMetricsResources` metric if the request is v0
`ListConfigResources`.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-06-03 02:04:04 +08:00
Chirag Wadhwa d1f41ef011
KAFKA-19320: Added share_consume_bench_test.py system tests (#19811)
This PR adds system tests in share_consume_bench_test.py for testing the
trogdor agent for Share Consumers/

Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Andrew
 Schofield <aschofield@confluent.io>
2025-06-02 15:38:56 +01:00
Apoorv Mittal a70a667e95
MINOR: Fixing logs and adding exception in response (#19859)
PR streamlines the logs when delete share group or offset is triggered.
Also fixes the response when group is not found while deleting share
group.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Sushant Mahajan
 <smahajan@confluent.io>
2025-06-02 15:10:51 +01:00
Andrew Schofield 223684bad1
KAFKA-16894: share.version becomes stable feature for preview (#19831)
This PR sets `SV_1` as the latest-production share version. This means
for AK 4.1 it will be a preview feature, not enabled by default, but can
be enabled without turning on unstable features. This is analogous to
how ELR worked for AK 4.0.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-02 13:45:37 +01:00
Chirag Wadhwa 942b11bb36
KAFKA-19321: Added share_consumer_performance.py and related system tests (#19836)
CI / build (push) Waiting to run Details
This PR includes some performance system tests utilizing the
kafka-share-consumer-perf.sh tool for share groups

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-02 10:55:50 +01:00
Kuan-Po Tseng 8d0097f60c
MINOR: fix failed StreamsBrokerBounceTest e2e tests (#19839)
Currently some tests in StreamsBrokersBounceTest failed due to error
`The cluster  does not support the STREAMS group protocol or does not
support the  versions of the STREAMS group protocol used by this client
(used  versions: 0 to 0).`

The reason is that under isolated kraft mode, we missed to set both
`unstable.api.versions.enable` and `unstable.feature.versions.enable` to
true to all controllers, which cause `streams.version` fallback to 0 in
the broker side and the above error raise when
StreamsGroupRequestHeartbeat comes to the broker.

This patch add the missing configs to controllers properties if streams
group protocol is used.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-02 10:40:59 +02:00
Jing-Jia Hung 8f1a1cd838
MINOR: Refactor awaitNonEmptyRecords to remove dead code and improve clarity (#19868)
This refactor improves the implementation of `awaitNonEmptyRecords` by:

- Removing the unreachable `throw new IllegalStateException` statement,
which was dead code due to `pollRecordsUntilTrue` throwing exceptions on
timeout.
- Eliminating the use of `return` inside the lambda, which relies on
non-local returns that can be confusing and error-prone in Scala.

Reviewers: Yung <yungyung7654321@gmail.com>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>

---------

Co-authored-by: Jing-Jia Hung <jing@Jing-JiadeiMac.local>
2025-06-02 12:46:56 +08:00
Hong-Yi Chen 8b49130b92
KAFKA-19355 Remove interBrokerListenerName from ClusterControlManager (#19866)
CI / build (push) Waiting to run Details
Following the removal of the ZK-to-KRaft migration code in commit
85bfdf4, controller-to-broker communication is now handled by the
control-plane listener (`controller.listener.names`). The
`interBrokerListenerName` parameter in `ClusterControlManager` is no
longer referenced on the controller side and can be safely removed as
dead code.

Reviewers: Lan Ding <isDing_L@163.com>, Ken Huang <s7133700@gmail.com>,
Chia-Ping Tsai <chia7712@gmail.com>
2025-06-02 01:18:15 +08:00
Ming-Yen Chung 6826f45fd8
KAFKA-19352 Create offsets topic to fix flaky testCommitAsyncCompletedBeforeConsumerCloses (#19873)
The flakiness occurs when the offsets topic does not yet exist. Hence,
the issue is mitigated by creating the offsets topic in `setup()`. This
serves as a workaround.  The root cause is tracked in
[KAFKA-19357](https://issues.apache.org/jira/browse/KAFKA-19357).

I ran the test 100 times on my Mac and all of them passed.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-02 01:01:24 +08:00
Hong-Yi Chen 77be6f2d74
KAFKA-19053 Remove FetchResponse#of which is not used in production … (#19327)
Removed the unused FetchResponse#of that is not used in production. The
test cases that originally invoked this method have been updated to call
the other

[FetchResponse#of](6af849f864/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java (L232)),
which is currently used by ```KafkaApis```, to maintain the integrity of
the tests.

Reviewers: Jun Rao <junrao@gmail.com>, PoAn Yang <payang@apache.org>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-06-02 00:48:53 +08:00
Logan Zhu 4eac6adf91
KAFKA-19284 Add documentation to clarify the behavior of null values for all partitionsToOffsetAndMetadata methods. (#19728)
CI / build (push) Waiting to run Details
Adds missing documentation to the `partitionsToOffsetAndMetadata`
methods in both `ListStreamsGroupOffsetsResult` and
`ListShareGroupOffsetsResult` classes to clarify the behavior when a
group does not have a committed offset for a specific partition.

As document in ListConsumerGroupOffsetsResult: > If the group doesn’t
have a committed offset for a specific partition, the corresponding
value in the returned map will be null.

This important detail was previously missing in the JavaDoc of the
stream and share group variants.

Reviewers: Nick Guo <lansg0504@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-01 02:17:47 +08:00
Ming-Yen Chung f9df9b5652
KAFKA-19311 Document commitAsync behavioral differences between Classic and Async Consumer (#19864)
#15613 ensures that all `commitAsync` callbacks are triggered before
`commitSync` completes for `AsyncKafkaConsumer`. However, the related
changes to `ClassicKafkaConsumer`, #15693, were not merged. I assume
this might be because we intend to gradually move toward using AsyncConsumer
instead.

In short, this behavioral difference should be documented.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-01 01:58:26 +08:00
Nick Guo cc0f06554b
KAFKA-19042 Move GroupAuthorizerIntegrationTest to clients-integration-tests module (#19685)
CI / build (push) Waiting to run Details
move GroupAuthorizerIntegrationTest to clients-integration-tests module

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, keemsisi <keemsisi@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-31 02:34:56 +08:00
Sean Quah 8323168b57
MINOR: Minor updates to RangeSet (#19678)
Minor updates to RangeSet:   * Disallow ranges with negative size   *
Disallow ranges with more than Integer.MAX_VALUE elements   * Fix
equals() so that all empty RangeSets are equal, to follow the Set
interface definition better.   * Reimplement hashCode() to follow the
Set interface definition.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-31 02:31:51 +08:00
Ken Huang 5e601b2b26
KAFKA-19042 Move BaseConsumerTest, SaslPlainPlaintextConsumerTest to client-integration-tests module (#19651)
Use Java to rewrite BaseConsumerTest, SaslPlainPlaintextConsumerTest by
new test infra and move it to client-integration-tests module, the
BaseConsumerTest test is still used, thus we should not remove now.

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-31 02:19:26 +08:00
Sanskar Jhajharia fafdf63a1d
MINOR: Add missing tag to testShareGroups test (#19860)
The commit 57ae6d6706 had mistakenly
removed the `@Test` tag from a test. Adding it back.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>, Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>
2025-05-30 15:02:01 +01:00
Lan Ding 7b99ee29a5
KAFKA-19346: Move LogReadResult to server module (#19846)
CI / build (push) Waiting to run Details
1. Move `LogReadResult` to server module.
2. Rewrite `LogReadResult` in Java.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-05-30 09:54:00 +01:00
Lan Ding 43f603cfb7
KAFKA-19351: AsyncConsumer#commitAsync should copy the input offsets (#19855)
`AsyncConsumer#commitAsync` and `AsyncConsumer#commitSync` should copy
the input offsets.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-30 09:36:38 +01:00
Nick Guo a122ac9d51
KAFKA-19042 move ConsumerWithLegacyMessageFormatIntegrationTest to clients-integration-tests module (#19810)
CI / build (push) Waiting to run Details
This PR rewrites `ConsumerWithLegacyMessageFormatIntegrationTest.scala`
in Java and moves it to the `clients-integration-tests module`.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-30 01:48:03 +08:00
PoAn Yang d1f1e5c8fd
KAFKA-18904: kafka-configs.sh return resource doesn't exist message [3/N] (#19808)
* Return resource doesn't exist message when users try to describe a
non-existent resource in kafka-configs.sh and kafka-client-metrics.sh.
* For groups type, the command checks both existent groups and
non-existent groups but having dynamic config. If it cannot find a group
in both conditions, return resource doesn't exist message.

Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Andrew
Schofield <aschofield@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-05-29 18:15:31 +01:00
Ritika Reddy f42abe6db8
KAFKA-19082:[4/4] Complete Txn Client Side Changes (KIP-939) (#19714)
public void completeTransaction(PreparedTxnState preparedTxnState)

The method compares the currently prepared transaction state and the
state passed in the argument.

1.  Commit if the state matches
2. Abort the transaction otherwise.

If the producer is not in a prepared state (i.e., neither
prepareTransaction was called nor initTransaction(true) was called), we
return an INVALID_TXN_STATE error.

Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
 <alivshits@confluent.io>
2025-05-29 09:06:57 -07:00
lorcan ef14f76fb3
KAFKA-6629: parameterise SegmentedCacheFunctionTest for session key schemas (#19404)
Addresses:
[KAFKA-6629](https://issues.apache.org/jira/browse/KAFKA-6629)

Adds configuration for the SessionKeySchema and parameterises the
existing tests  so that both WindowKeys and SessionKeys are tested under
the existing unit tests.

Reviewers: Bill Bejeck <bbejeck@apache.org>

---------

Co-authored-by: Lorcan <lorcanjames1@gmail.com>
2025-05-29 11:57:10 -04:00
Sushant Mahajan 13b5627274
KAFKA-19337: Write state writes snapshot for higher state epoch. (#19843)
- Due to condition on number of updates/snapshot in
`generateShareStateRecord`, share updates gets written for write state
requests even if they have the highest state epoch seen so far.
- A share update cannot record state epoch. As a result, this update
gets missed.
- This PR remedies the issue and adds a test as proof of the fix.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-29 13:45:54 +01:00
Andrew Schofield bd939f56de
MINOR: Removed stale comment (#19848)
Removed a trivial stale comment.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-29 13:44:11 +01:00
Ming-Yen Chung aafc1ae27e
KAFKA-19056 Rewrite EndToEndClusterIdTest in Java and move it to the server module (#19741)
Use ClusterTest and java to rewrite `EndToEndClusterIdTest` and move it
to the server module

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-29 19:08:05 +08:00
Chirag Wadhwa 64b63f472f
KAFKA-19316: added share_group_command_test.py system tests (#19774)
CI / build (push) Waiting to run Details
This PR include system tests in the file share_group_command_test.py.
These tests test the functionality of kafka-share-groups.sh tool

Reviewers: Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-05-29 10:59:32 +01:00
Chirag Wadhwa 2133b5e3f4
MINOR: removed test cases with combined_kraft mode in share_consumer_test.py (#19841)
This PR removes the test cases from share_consumer_test.py where
combined_kraft mode is used

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-29 10:23:14 +01:00
Kevin Wu 8731c96122
MINOR: fixing updateBrokerContactTime (#19828)
Fix `updateBrokerContactTime` so that existing brokers still have their
contact time updated when they are already tracked. Also, update the
unit test to test this case.

Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Yung
 <yungyung7654321@gmail.com>, TengYao Chi <frankvicky@apache.org>, Ken
 Huang <s7133700@gmail.com>
2025-05-29 11:58:09 +08:00
Sushant Mahajan 383a9ff9df
KAFKA-19344: Replace desc.assignablePartitions with spec.isPartitionAssignable. (#19838)
CI / build (push) Waiting to run Details
- A new method `assignablePartitions` was added to the
`SubscribedTopicDescriber`in https://github.com/apache/kafka/pull/19026.
This method was required for computing assignments for share groups
(KIP-932).
- However, since the describer is a public interface and is used to
encapsulate methods which return all subscribed partitions (KIP-848),
`assignablePartitions` is deemed inconsistent with this interface.
- Hence, this PR extends the `GroupSpec` interface to add a method
`isPartitionAssignable` which will serve the same purpose. The
`assignablePartitions` has been removed from the describer.
- Tests have been updated for the assigners and spec and removed from
describer as required.

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Jacot
 <djacot@confluent.io>
2025-05-28 20:27:29 +01:00
David Arthur 9dd4cff2d7
KAFKA-19347 Don't update timeline data structures in createAcls (#19840)
CI / build (push) Waiting to run Details
This patch fixes a problem in AclControlManager where we are updating
the timeline data structures prematurely.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>, Andrew Schofield <aschofield@confluent.io>,
2025-05-28 09:40:19 -07:00
Andrew Schofield be4762e401
KAFKA-19342: Authorization tests for alter share-group offsets (#19826)
Add integration tests for alter share group offsets API.

Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Sushant
 Mahajan <smahajan@confluent.io>, Apoorv Mittal
 <apoorvmittal10@gmail.com>
2025-05-28 16:33:18 +01:00
Lucas Brutschy 0c116c9629
KAFKA-19335: Membership managers send negative epoch in JOINING (#19818)
There is a sequence of interactions with the membership managers of
KIP-848, KIP-932, KIP-1071 that can put the membership manager into
JOINING state, but where member epoch is set to -1. This can result in
an invalid request being sent, since joining heartbeats should not have
member epoch -1. This may lead to the member failing to join. In the
case of streams, the group coordinator will return INVALID_REQUEST.

This is the sequence triggering the bug, which seems to relatively
likely, caused by two heartbeat responses being received after the next
one has been sent.

```
membershipManager.leaveGroup();
    -> transitions to LEAVING
membershipManager.onHeartbeatRequestGenerated();
    -> transitions to UNSUBSCRIBED
membershipManager.onHeartbeatSuccess(... with member epoch > 0);
    -> unblocks the consumer
membershipManager.onSubscriptionUpdated();
membershipManager.onConsumerPoll();
    -> transitions to JOINING
membershipManager.onHeartbeatSuccess(... with member epoch < 0);
    -> updates the epoch to a negative value
```
Now we are in state `JOINING` with `memberEpoch=-1`, and the next
heartbeat we send will be malformed, triggering `INVALID_REQUEST`

The bug may also be triggered if the `unsubscribe` times out, but this
seems more of a corner case.

To prevent the bug, we are taking two measures: The likely path to
triggering the bug can be prevented by not unblocking an `unsubscribe`
call in the consumer when a non-leave-heartbeat epoch is received. Once
we have sent out leave group heartbeat, we will ignore all heartbeats,
except for those containing `memberEpoch < 0`.

For extra measure, we also prevent the second case (`unsubscribe` timing
out). In this case, the consumer gets unblocked before we have received
the leave group heartbeat response, and may resubscribe to the group. In
this case, we shall just ignore the heartbeat response that contains a
member epoch < 0, once it arrives, and we have already left the
`UNSUBSCRIBED` state.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>, Shivsundar R <shr@confluent.io>
2025-05-28 16:47:00 +02:00
Chirag Wadhwa c398261834
KAFKA-19285: Added more tests in SharePartitionManagerTest (#19778)
This PR includes more unit tests in the file SharePartitionManagerTest
to improve the coverage

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-28 15:15:56 +01:00
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