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>
This PR includes the system test file test_console_share_consumer.py
which tests the functioning of ConsoleShareConsumer
Reviewers: Andrew Schofield <aschofield@confluent.io>
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>
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>
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>
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>
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>
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>
* 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>
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>
### 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>
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>
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>
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>
* 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>
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>
* 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>
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>
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>
* 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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
Guard against possible `NullPointerExceptions` in
`ConsumerNetworkThread.cleanup()` if
`ConsumerNetworkThread.initializeResources()` previously failed.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
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>
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>
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>
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>
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>
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>
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>
* 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>
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>
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>
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>
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>
* 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>
At the retry limit binaryExponentialElectionBackoffMs it becomes
statistically likely that the exponential backoff returned
electionBackoffMaxMs. This is an issue as multiple replicas can get
stuck starting elections at the same cadence.
This change fixes that by added a random jitter to the max election
backoff.
Reviewers: José Armando García Sancio <jsancio@apache.org>, TaiJuWu
<tjwu1217@gmail.com>, Yung <yungyung7654321@gmail.com>