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>
As a result of KAFKA-18905 the reassign test will often have test
failures which are unrelated to the actual reassignment of partitions.
This failure is mentioned in KAFKA-9199.
Quote from KAFKA-9199: "This issue popped up in the reassignment system
test. It ultimately caused the test to fail because the producer was
stuck retrying the duplicate batch repeatedly until ultimately giving
up."
Disabling the idempotent producer circumvents this issue and allows the
reassignment system tests to succeed reliably. The reassignment test
still check that produce batches were not lost.
Reviewers: José Armando García Sancio <jsancio@apache.org>
As part of KIP-1043, `Admin.listConsumerGroups()` and variants have been
deprecated. This is because there are now 4 types of group and listing
has been consolidated under `Admin.listGroups()`. This PR adds the
deprecation information to the upgrade documentation.
Reviewers: Lianet Magrans <lmagrans@confluent.io>
### Motivation
While investigating “events skipped in group
rebalancing” ([spring‑projects/spring‑kafka#3703](https://github.com/spring-projects/spring-kafka/issues/3703))
I discovered a race
condition between
- the main poll/commit thread, and
- the consumer‑coordinator heartbeat thread.
If the main thread enters
`ConsumerCoordinator.sendOffsetCommitRequest()` while the heartbeat
thread is finishing a rebalance (`SyncGroupResponseHandler.handle()`),
the group state transitions in the following order:
```
COMPLETING_REBALANCE → (race window) → STABLE
```
Because we read the state twice without a lock:
1. `generationIfStable()` returns `null` (state still
`COMPLETING_REBALANCE`),
2. the heartbeat thread flips the state to `STABLE`,
3. the main thread re‑checks with `rebalanceInProgress()` and wrongly
decides that a rebalance is still active,
4. a spurious `CommitFailedException` is returned even though the commit
could succeed.
For more details, please refer to sequence diagram below. <img
width="1494" alt="image"
src="https://github.com/user-attachments/assets/90f19af5-5e2d-4566-aece-ef764df2d89c"
/>
### Impact
- The exception is semantically wrong: the consumer is in a stable
group, but reports failure.
- Frameworks and applications that rely on the semantics of
`CommitFailedException` and `RetryableCommitException` (for example
`Spring Kafka`) take the wrong code path, which can ultimately skip the
events and break “at‑most‑once” guarantees.
### Fix
We enlarge the synchronized block in
`ConsumerCoordinator.sendOffsetCommitRequest()` so that the consumer
group state is examined atomically with respect to the heartbeat thread:
### Jira
https://issues.apache.org/jira/browse/KAFKA-19242
https: //github.com/spring-projects/spring-kafka/issues/3703
Signed-off-by: chickenchickenlove <ojt90902@naver.com>
Reviewers: David Jacot <david.jacot@gmail.com>
When fetching stable offsets in the group coordinator, we iterate over
all requested partitions. For each partition, we iterate over the
group's ongoing transactions to check if there is a pending
transactional offset commit for that partition.
This can get slow when there are a large number of partitions and a
large number of pending transactions. Instead, maintain a list of
pending transactions per partition to speed up lookups.
Reviewers: Shaan, Dongnuo Lyu <dlyu@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jaco <djacot@confluent.io>
Currently, the quorum uses kraft by default, so there's no need to
specify it explicitly.
For kip932 and isShareGroupTest, they are no longer used after #19542 .
Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The fallback mechanism for `remote.log.manager.copier.thread.pool.size`
and `remote.log.manager.expiration.thread.pool.size` defaulting to
`remote.log.manager.thread.pool.size` was introduced in KIP-950. This
approach was abandoned in KIP-1030, where default values were changed
from -1 to 10, and a configuration validator enforcing a minimum value
of 1 was added. As a result, this commit removes the fallback mechanism
from `RemoteLogManagerConfig.java` to align with the new defaults and
validation.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Migrates the `TxnTransitMetadata` class from scala to java, moving it
from to the `transaction-coordinator` module.
Reviewers: PoAn Yang <payang@apache.org>, Nick Guo
<lansg0504@gmail.com>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
jira: https://issues.apache.org/jira/browse/KAFKA-19068
`RecordsIterator#decodeControlRecord` do the type check and then
`ControlRecord` constructor does that again.
we should add a static method to ControlRecord to create `ControlRecord`
with type check, and then `ControlRecord` constructor should be changed
to private to ensure all instance is created by the static method.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
If directory id is `Uuid.ZERO_UUID`, the command don't print the result.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Add tests to check whether the results returned by the API
`createTopics` and `describeConfigs` contain internal configurations.
Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>, TaiJuWu
<tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Retaining a reference to ```AbstractConfig``` introduced coupling and
potential inconsistencies with dynamic config updates. This change
simplifies ```TransactionLogConfig``` into a POJO by removing the
internal ```AbstractConfig``` field, and aligns with feedback from
#19439
Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
[As discussed in the mailing
list](https://lists.apache.org/thread/m03mpkm93737kk6d1nd6fbv9wdgsrhv9),
the broker only fetches remote data for ONE partition in a given
FetchRequest. In other words, if a consumer sends a FetchRequest
requesting 50 topic-partitions, and each partition's requested offset is
not stored locally - the broker will fetch and respond with just one
partition's worth of data from the remote store, and the rest will be
empty.
Given our defaults for total fetch response is 50 MiB and per partition
is 1 MiB, this can limit throughput. This patch documents the behavior
in 3 configs - `fetch.max.bytes`, `max.partition.fetch.bytes` and
`remote.fetch.max.wait.ms`
Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash
<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
@xijiu's invitation is timeout, so we have to remove the name, then
re-add it in a new commit.
see https://issues.apache.org/jira/browse/INFRA-26796
Reviewers: PoAn Yang <payang@apache.org>, xijiu <422766572@qq.com>,
Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ken Huang <s7133700@gmail.com>,
TengYao Chi <frankvicky@apache.org>
- A couple of newly added tests were found to be flaky in
`AuthorizerIntegrationTest.scala`.
- `testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl` and
`testShareGroupDescribeWithoutGroupDescribeAcl`. These tests pass
locally, so could not replicate the failure.
- But logs from develocity indicated that the test fails when the
following condition happens :
When the background error event arrives after the consumer had
unsubscribed, then these events are processed in the
`handleCompletedAcknowledgements` method and the exception from the
event is thrown, preventing `close()` to complete.
- We need to handle this race condition where we might get the
background event after unsubscribe and before processing the callbacks.
- PR fixes this by ignoring the exceptions in the background queue when
the `handleCompletedAcknowledgements` method is called during `close()`.
This ensures `close()` completes successfully.
- Have added a unit test which mimics the race condition as well.
Reviewers: Andrew Schofield <aschofield@confluent.io>
Admin.listConsumerGroups() was able to use the early versions of
ListGroups RPC with the version used dependent upon the filters the user
specified. Admin.listGroups(ListGroupsOptions.forConsumerGroups())
inadvertently required ListGroups v5 because it always set a types
filter. This patch handles the UnsupportedVersionException and winds
back the complexity of the request unless the user has specified filters
which demand a higher version.
It also adds ListGroupsOptions.forShareGroups() and forStreamsGroups().
The usability of Admin.listGroups() is much improved as a result.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang
<payang@apache.org>
Update docs to note index.interval.bytes sets entry frequency for offset index and, conditionally, time index. Improve clarity and readability of index.interval.bytes description.
Reviewers: Luke Chen <showuon@gmail.com>
When performing perf tests, we can specify a payload using the
"--payloadFile" flag. This file is utilized during the load/performance
testing process. This causes the entire file to get loaded into a String
and split using the delimiter. However, if the file is large, it may
result in NegativeArraySizeException error.
Moving the file loading logic to Scanner which doesn't have this issue.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Ken Huang
<s7133700@gmail.com>, Zhe Guang <zheguang.zhao@alumni.brown.edu>
@xijiu's invitation is timeout, so we have to remove the name, then
re-add it in a new commit.
see https://issues.apache.org/jira/browse/INFRA-26796
Reviewers: Chih-Yuan Chien <joshua2519@gmail.com>, Kuan-Po Tseng
<brandboat@gmail.com>, PoAn Yang <payang@apache.org>, yunchi
<yunchipang@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Hong-Yi Chen
<apalan60@gmail.com>, Bolin Lin <linbolin1230@gmail.com>, Shih-Yuan Lin
<shmily7829@gmail.com>, Mirai1129 <minecraftmiku831@gmail.com>
This PR moves SchedulerTest to server module and rewrite it with java.
Please also check updated import control config!
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
While working on https://github.com/apache/kafka/pull/19515, I came to
the conclusion that the OffsetFetchResponse is quite messy and overall
too complicated. This patch rationalize the constructors.
OffsetFetchResponse has a single constructor accepting the
OffsetFetchResponseData. A builder is introduced to handle the down
conversion. This will also simplify adding the topic ids. All the
changes are mechanical, replacing data structures by others.
Reviewers: Lianet Magrans <lmagrans@confluent.io>
The PR adds `slice` API in `Records.java` and further implementation in
`MemoryRecords`. With the addition of ShareFetch and it's support to
read from TieredStorage, where ShareFetch might acquire subset of fetch
batches and TieredStorage emits MemoryRecords, hence a slice API is
needed for MemoryRecords as well to limit the bytes transferred (if
subset batches are acquired).
MemoryRecords are sliced using `duplicate` and `slice` API of
ByteBuffer, which are backed by the original buffer itself hence no-copy
is created rather position, limit and offset are changed as per the new
position and length.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao
<junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The current ElectionWasClean checks if the new leader is in the previous
ISR. However, there is a corner case in the partition reassignment. The
partition reassignment can change the partition replicas. If the new
preferred leader (the first one in the new replicas) is the last one to
join ISR, this preferred leader will be elected in the same partition
change.
For example: In the previous state, the partition is Leader: 0,
Replicas (2,1,0), ISR (1,0), Adding(2), removing(0). Then replica 2
joins the ISR. The new partition would be like: Leader: 2, Replicas
(2,1), ISR(1,2). The new leader 2 is not in the previous ISR (1,0) but
it is still a clean election.
Reviewers: Jun Rao <junrao@gmail.com>
- Add new DS to wrap the member assignment (containing topic Ids, names
and partitions), to easily access the data as needed. This will be used
in following PR to integrate assignment with topic IDs into the
subscription state.
- Improve logging on the client assignment/reconciliation path
No changes in logic.
Reviewers: TengYao Chi <frankvicky@apache.org>, Andrew Schofield
<aschofield@confluent.io>
Change:
`public List<Optional<? extends RequestManager>> entries();`
to:
`public List<RequestManager> entries();`
and clean up the callers.
Reviewers: TengYao Chi <kitingiao@gmail.com>, Andrew Schofield
<aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This PR adds an additional test case to `FormatterTest` that checks that
formatting with `--standalone` and then formatting again with
`--standalone --ignore-formatted` is indeed a no-op.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Currently in the tests in ShareFetchAcknowledgeRequestTest, subsequent
share fetch / share acknowledge requests creates a new socket everytime,
even when the requests are sent by the same member. In reality, a single
share consumer clisnet will reuse the same socket for all the share
related requests in its lifetime. This PR changes the behaviour in the
tests to align with reality and reuse the same socket for all requests
by the same share group member.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
`ConstantBrokerOrActiveKController` was introduced in #14399, to provide
a mechanism for selecting the least loaded broker or the active
controller when using `bootstrap.controllers`.
Usage was removed in #18002, after `alterConfigs` was deprecated in
Kafka 2.4.0.
Reviewers: PoAn Yang <payang@apache.org>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
### About
Added code to handle share partition fetch lock cleanly in
`DelayedShareFetch` to avoid a member incorrectly releasing a share
partition's fetch lock
### Testing
The code has been tested with the help of unit tests and integration
tests.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
New system test for KIP-1071.
Standby replicas need to be enabled via `kafka-configs.sh`.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax
<matthias@confluent.io>
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Kirk True <kirk@kirktrue.pro>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ian McDonald <ian_mcdonald@rocketmail.com>
Log a message when reading a batch that is larger than the currently
allocated batch.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, PoAn Yang
<payang@apache.org>
When the persister is responding to a read share-group state summary
request, it has no way of including the leader epoch in its response,
even though it has the information to hand. This means that the leader
epoch information is not initialised in the admin client operation to
list share group offsets, and this then means that the information
cannot be displayed in kafka-share-groups.sh.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan
<smahajan@confluent.io>
- Removed unused fields and methods in clients tests
- Fixed IDEA code inspection warnings
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Andrew Schofield <aschofield@confluent.io>,
Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
<frankvicky@apache.org>
1. Move `ForwardingManagerMetrics` and `ForwardingManagerMetricsTest` to
server module.
2. Rewrite them in Java.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
rewrite `MetricsDuringTopicCreationDeletionTest` to `ClusterTest` infra
and move it to clients-integration-tests module.
Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
This PR is fixing the issue introduced in #19120
The input `StreamsGroup`-options must not be ignored, but it must be
converted to `ConsumerGroup`-options.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
The output from the delete-offsets option of kafka-consumer-groups.sh
can be improved. For example, the column widths are excessive which
looks untidy, and the output messages can be improved.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
The PR fixes the issue when ShareAcknowledgements are piggybacked on
ShareFetch. The current default configuration in clients sets `batch
size` and `max fetch records` as per the `max.poll.records` config,
default 500. Which means all records in a single poll will be fetched
and acknowledged. Also the default configuration for inflight records in
a partition is 200. Which means prior fetch records has to be
acknowledged prior fetching another batch from share partition.
The piggybacked share fetch-acknowledgement calls from KafkaApis are
async and later the response is combined. If respective share fetch
starts waiting in purgatory because all inflight records are currently
full, hence when startOffset is moved as part of acknowledgement, then a
trigger should happen which should try completing any pending share
fetch requests in purgatory. Else the share fetch requests wait in
purgatory for timeout though records are available, which dips the share
fetch performance.
The regular fetch has a single criteria to land requests in purgatory,
which is min bytes criteria, hence any produce in respective topic
partition triggers to check any pending fetch requests. But share fetch
can wait in purgatory because of multiple reasons: 1) Min bytes 2)
Inflight records exhaustion 3) Share partition fetch lock competition.
The trigger already happens for 1 and current PR fixes 2. We will
investigate further if there should be any handling required for 3.
Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield
<aschofield@confluent.io>
In https://github.com/apache/kafka/pull/16578 , we tried to exclude both
`checker-qual` and `error_prone_annotations`, but when excluding
`error_prone_annotations`, the compilation failed. So in the end, we
only excluded `checker-qual` and shipped `error_prone_annotations.jar`
to users. In Kafka v4.0.0, thanks to jdk 8 removal, we upgraded caffeine
to the latest v3.1.8, instead of v2.x.x, and now, we can successfully
pass the compilation without error after excluding
`error_prone_annotations` from `caffeine`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang <s7133700@gmail.com>
The test is failing once in a while but there is not enough information
in the logs to determine the root cause. Adding more information, and
fixing thread resource leak.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
### About
11 of the test cases in `SharePartitionTest` have failed at least once
in the past 28 days.
https://develocity.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.timeZoneId=Europe%2FLondon&tests.container=kafka.server.share.SharePartitionTest
Observing the flakiness, they seem to be caused due to the usage of
`SystemTimer` for various acquisition lock timeout related tests. I have
replaced the usage of `SystemTimer` with `MockTimer` and also improved
the `MockTimer` API with regard to removing the timer task entries that
have already been cancelled.
Also, this has reduced the time taken to run `SharePartitionTest` from
~6 sec to ~1.5 sec
### Testing
The testing has been done with the help of already present unit tests in
Apache Kafka.
Reviewers: Andrew Schofield <aschofield@confluent.io>
### About
This PR removes the limitation in remote storage fetch for share groups
of only performing remote fetch for a single topic partition in a share
fetch request. With this PR, share groups can now fetch multiple remote
storage topic partitions in a single share fetch request.
### Testing
I have followed the [AK
documentation](https://kafka.apache.org/documentation/#tiered_storage_config_ex)
to test my code locally (by adopting `LocalTieredStorage.java`) and
verify with the help of logs that remote storage is happening for
multiple topic partitions in a single share fetch request. Also,
verified it with the help of unit tests.
Reviewers: Jun Rao <junrao@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
Add a wait for cleaner thread shutdown in `testCleanerThreadShutdown` to
eliminate flakiness. After calling `cache.close()`, the test now uses
`TestUtils.waitForCondition` to poll until the background
“remote-log-index-cleaner” thread has fully exited before asserting that
no cleaner threads remain. This ensures the asynchronous shutdown always
completes before the final assertions.
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
The PR do following:
1. Move MetadataVersionIntegrationTest to clients-integration-tests
module
2. rewrite to java from scala
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>