* Add `group.share.assignors` config to `GroupCoordinatorConfig`.
* Send `rackId` in share group heartbeat request if it's not null.
* Add integration test `testShareConsumerWithRackAwareAssignor`.
Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Andrew
Schofield <aschofield@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
The mapKey optimisation can be used in some KIP-932 RPC schemas to
improve efficiency of some key-based accesses.
* AlterShareGroupOffsetsResponse
* ShareFetchRequest
* ShareFetchResponse
* ShareAcknowledgeRequest
* ShareAcknowledgeResponse
Reviewers: Andrew Schofield <aschofield@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
Updated the code to start the State Updater Thread only after the Stream
Thread is started.
Changes done :
1. Moved the starting of the StateUpdater thread to a new init method in
the TaskManager.
2. Called the init of TaskManager in the run method of the StreamThread.
3. Updated the test cases in the StreamThreadTest to mimic the
aforementioned behaviour.
Reviewers: Bruno Cadonna <cadonna@apache.org>
This PR simplifies two ConcurrentHashMap fields by removing their Atomic
wrappers:
- Change `brokerContactTimesMs` from `ConcurrentHashMap<Integer,
AtomicLong>` to `ConcurrentHashMap<Integer, Long>`.
- Change `brokerRegistrationStates` from `ConcurrentHashMap<Integer,
AtomicInteger>` to `ConcurrentHashMap<Integer, Integer>`.
This removes mutable holders without affecting thread safety (see
discussion in #19828).
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Kevin Wu <kevin.wu2412@gmail.com>, Ken Huang
<s7133700@gmail.com>
As part of readying share groups for production, we want to ensure that
the performance of the server-side assignor is optimal. In common with
consumer group assignors, a JMH benchmark is used for the analysis.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
Now that Kafka Brokers support Java 17, this PR makes some changes in
core module. The changes in this PR are limited to only some Scala files
in the Core module's tests. 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 from unit.kafka
module:
- log
- network
- security
- tools
- utils
Reviewers: TengYao Chi <frankvicky@apache.org>
The PR do following:
1. rewrite to new test infra
2. rewrite to java
3. move to clients-integration-tests
Reviewers: Ken Huang <s7133700@gmail.com>, Kuan-Po Tseng
<brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
- Moving off deprecated methods
- Fixing argument order for assertEquals(...)
- Few other minor cleanups
Reviewers: PoAn Yang <payang@apache.org>, Lianet Magrans
<lmagrans@confluent.io>, Ken Huang <s7133700@gmail.com>
This PR is part of the KIP-1034.
It brings the support for the source raw key and the source raw
value in the `ErrorHandlerContext`. Required by the routing to DLQ implemented
by https://github.com/apache/kafka/pull/17942.
Reviewers: Bruno Cadonna <cadonna@apache.org>
Co-authored-by: Damien Gasparina <d.gasparina@gmail.com>
Fix to ensure protocol name comparison in integration test ignore case
(group protocol from param is lower case, vs enum name upper case)
The tests were not failing but the custom configs/expectation were not
being applied depending on the protocol (the tests checks for
"groupProtocol.equals(CLASSIC)" would never be true.
Found all comparisons with equals agains the constant name and fixed
them (not too many luckily).
I did consider changing the protocol param that is passed to every test
(that is now lowercase), but still, seems more robust to have the tests
ignore case.
Reviewers: Gaurav Narula <gaurav_narula2@apple.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
<frankvicky@apache.org>
With the transaction V2, replica manager checks whether the incoming
producer request produces to a partition belonging to a transaction.
ReplicaManager figures this out by checking the producer epoch stored in
the partition log. However, the current code does not reject the produce
request if its producer epoch is lower than the stored producer epoch.
It is an optimization to reject such requests earlier instead of sending
an AddPartitionToTxn request and getting rejected in the response.
Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
<alivshits@confluent.io>
We should behave more like a consumer group and make sure to not be
subscribed to the input topics anymore when the last member leaves the
group. We don't do this right now because our topology is still
initialized even after the last member leaves the group.
This will allow:
* Offsets to expire and be cleaned up.
* Offsets to be deleted through admin API calls.
Reviewers: Bill Bejeck <bbejeck@apache.org>
A heartbeat might be sent to the group coordinator, claiming to own
tasks that we do not know about. We need some logic to handle those
requests. In KIP-1071, we propose to return `INVALID_REQUEST` error
whenever this happens, effectively letting the clients crash.
This behavior will, however, make topology updates impossible. Bruno
Cadonna proposed to only check that owned tasks match our set of
expected tasks if the topology epochs between the group and the client
match. The aim of this change is to implement a check and a behavior
for the first version of the protocol, which is to always return
`INVALID_REQUEST` if an unknown task is sent to the group coordinator.
We can relax this constraint once we allow topology updating with
topology epochs.
To efficiently check this whenever we receive a heartbeat containing
tasks, we precompute the number of tasks for each subtopology. This also
benefits the performance of the assignor.
Reviewers: Bill Bejeck <bbejeck@apache.org>
* Add `RackAwareAssignor`. It uses `racksForPartition` to check the rack
id of a partition and assign it to a member which has the same rack id.
* Add `ConsumerIntegrationTest#testRackAwareAssignment` to check
`racksForPartition` works correctly.
Reviewers: David Jacot <djacot@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
In #19840, we broke de-duplication during ACL creation. This patch fixes
that and adds a test to cover this case.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Hi, I've created pull request.
jira: [19328](https://issues.apache.org/jira/browse/KAFKA-19328)
problem:
1. doAnswer chaining works as intended only when calls are made
sequentially. In a multithreaded environment, its behavior is
unpredictable.
2. errors in a thread can be swallowed, not seen in main thread.
3. 5 doAnswer chain is not enough for 100 threads. The last chain is
returned for most cases.
4. nextFetchOffset seems to be called before doAnswer chain, so the last
values (25, 5, 26, 16) always was found in doAsnwer chain.
solution:
Delete doAnswer chain so that above four problems disappear.
Reviewers: Abhinav Dixit <adixit@confluent.io>, Apoorv Mittal
<apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
This PR implements all the options for `--delete --group grpId` and
`--delete --all-groups`
Tests: Integration tests and unit tests.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Andrew Schofield
<aschofield@confluent.io>
The `LIST_CLIENT_METRICS_RESOURCES` RPC was generalised to all config
resources in AK 4.1 and the RPC was renamed to `LIST_CONFIG_RESOURCES`.
This PR updates the RPC authorisation table in the documentation.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
### About
Replaced `.close` functionality with `try-with-resources` for few tests
in `DelayedShareFetchTest` where we required to use `mockStatic`.
### Testing
The code has been tested by running the unit tests.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
Adding support for the `urn:ietf:params:oauth:grant-type:jwt-bearer`
grant type (AKA `jwt-bearer`). Includes further refactoring of the
existing OAuth layer and addition of generic JWT assertion layer that
can be leveraged in the future.
This constitutes the main piece of the JWT Bearer grant type support.
Forthcoming commits/PRs will include improvements for both the
`client_credentials` and `jwt-bearer` grant types in the following
areas:
* Integration test coverage (KAFKA-19153)
* Unit test coverage (KAFKA-19308)
* Top-level documentation (KAFKA-19152)
* Improvements to and documentation for `OAuthCompatibilityTool`
(KAFKA-19307)
Reviewers: Manikumar Reddy <manikumar@confluent.io>, Lianet Magrans
<lmagrans@confluent.io>
---------
Co-authored-by: Zachary Hamilton <77027819+zacharydhamilton@users.noreply.github.com>
Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
This PR adds integration tests for `--list`
(Transferred from the feature branch `kip1071`) related ticket:
[KAFKA-18887](https://issues.apache.org/jira/browse/KAFKA-18887)
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
The `String.split` method never returns an array containing null
elements.
Reviewers: TengYao Chi <frankvicky@apache.org>, Ken Huang
<s7133700@gmail.com>, Lan Ding <isDing_L@163.com>
## Problem
When an `txnProducer.abortTransaction()`operation encounters a
`TRANSACTION_ABORTABLE` error, it currently tries to transition to
`ABORTABLE_ERROR` state. This can create an infinite retry loop since:
1. The abort operation fails with `TRANSACTION_ABORTABLE`
2. We transition to `ABORTABLE_ERROR` state
3. The application recieves instance of TransactionAbortableException
and it retries the abort
4. The cycle repeats
## Solution
For `txnProducer.abortTransaction()`API, convert
`TRANSACTION_ABORTABLE` errors to fatal errors (`KafkaException`) during
abort operations to ensure clean transaction termination. This prevents
retry loops by:
1. Treating abort failures as fatal errors at application layer
2. Ensuring the transaction can be cleanly terminated
3. Providing clear error messages to the application
## Changes
- Modified `EndTxnHandler.handleResponse()` to convert
`TRANSACTION_ABORTABLE` errors to `KafkaException` during abort
operations
- Set TransactionManager state to FATAL
- Updated test `testAbortableErrorIsConvertedToFatalErrorDuringAbort` to
verify this behavior
## Testing
- Added test case verifying that abort operations convert
`TRANSACTION_ABORTABLE` errors to `KafkaException`
- Verified that Commit API with TRANSACTION_ABORTABLE error should
set TM to Abortable state
- Verified that Abort API with TRANSACTION_ABORTABLE error should
convert to Fatal error i.e. KafkaException
## Impact
At application layer, this change improves transaction reliability by
preventing infinite retry loops during abort operations.
Reviewers: Justine Olshan <jolshan@confluent.io>
### 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>
* 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>
Update catch to handle compression errors
Before :

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>
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>
* 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>
- 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>
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>
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>
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>
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>
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>
This PR includes some performance system tests utilizing the
kafka-share-consumer-perf.sh tool for share groups
Reviewers: Andrew Schofield <aschofield@confluent.io>
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>
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>
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>
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>
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>
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>
#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>
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>
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>
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>