https://issues.apache.org/jira/browse/KAFKA-19383 When applying the
ClearElrRecord, it may pick up the topicId in the image without checking
if the topic has been deleted. This can cause the creation of a new
TopicRecord with an old topic ID.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
No conflicts.
Previously, we could wait for up to half of the broker session timeout
for an RPC to complete, and then delay by up to half of the broker
session timeout. When taken together, these two delays could lead to
brokers erroneously missing heartbeats.
This change removes exponential backoff for heartbeats sent from the
broker to the controller. The load caused by heartbeats is not heavy,
and controllers can easily time out heartbeats when the queue length is
too long. Additionally, we now set the maximum RPC time to the length of
the broker period. This minimizes the impact of heavy load.
Reviewers: José Armando García Sancio <jsancio@apache.org>, David Arthur <mumrah@gmail.com>
If there are more deletion filters after we initially hit the
`MAX_RECORDS_PER_USER_OP` bound, we will add an additional deletion
record ontop of that for each additional filter.
The current error message returned to the client is not useful either,
adding logic so client doesn't just get `UNKNOWN_SERVER_EXCEPTION` with
no details returned.
## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.
Reviewers: Vincent PÉRICART <mauhiz@gmail.com>, Jun Rao
<junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This is a follow up to
[https://github.com/apache/kafka/pull/19910](https://github.com/apache/kafka/pull/url)
The coordinator failed to write an epoch fence transition for producer
jt142 to the transaction log with error COORDINATOR_NOT_AVAILABLE. The
epoch was increased to 2 but not returned to the client
(kafka.coordinator.transaction.TransactionCoordinator) -- as we don't
bump the epoch with this change, we should also update the message to
not say "increased" and remove the
epochAndMetadata.transactionMetadata.hasFailedEpochFence = true line
In the test, the expected behavior is:
First append transaction to the log fails with
COORDINATOR_NOT_AVAILABLE (epoch 1)
We try init_pid again, this time the SINGLE epoch bump succeeds, and
the following things happen simultaneously (epoch 2)
-> Transition to COMPLETE_ABORT
-> Return CONCURRENT_TRANSACTION error to the client
The client retries, and there is another epoch bump; state
transitions to EMPTY (epoch 3)
Reviewers: Justine Olshan <jolshan@confluent.io>
[0b2e410d61](url)
Bug fix in 4.0
**Conflicts:**
- The Transaction Coordinator had some conflicts, mainly with the
transaction states. Ex: ongoing in 4.0 is TransactionState.ONGOING in
4.1.
- The TransactionCoordinatorTest file had conflicts w.r.t the 2PC
changes from KIP-939 in 4.1 and the above mentioned state changes
Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
<alivshits@confluent.io>
Minor fix to correct the validate condition for GetTelemetryRequests.
Added respective tests as well.
Reviewers: Andrew Schofield <aschofield@confluent.io>
## Summary
- MetadataShell may deletes lock file unintentionally when it exists or
fails to acquire lock. If there's running server, this causes unexpected
result as below:
* MetadataShell succeeds on 2nd run unexpectedly
* Even worse, LogManager/RaftManager's lock also no longer work from
concurrent Kafka process startup
Reviewers: TengYao Chi <frankvicky@apache.org>
# Conflicts:
# shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java
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>
### 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>
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>
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>
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>
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>
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>
### 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>
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 remote storage reader thread pool use same count for both maximum
and core size. If users adjust the pool size larger than original value,
it throws `IllegalArgumentException`. Updated both value to fix the
issue.
cherry-pick PR: #19532
cherry-pick commit:
965743c35b
---------
Signed-off-by: PoAn Yang <payang@apache.org>
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang
<payang@apache.org>
Co-authored-by: PoAn Yang <payang@apache.org>
Cherry-picked from
[e79f5f0](e79f5f0f65)
If a share or consumer group is described, all group IDs sent to all
shards of the group coordinator. This change fixes it. It tested in the
unit tests, since it's somewhat inconvenient to test the passed read
operation lambda.
Old bootstrap.metadata files cause problems with server that include
KAFKA-18601. When the server tries to read the bootstrap.checkpoint
file, it will fail if the metadata.version is older than 3.3-IV3
(feature level 7). This causes problems when these clusters are
upgraded.
This PR makes it possible to represent older MVs in BootstrapMetadata
objects without causing an exception. An exception is thrown only if we
attempt to access the BootstrapMetadata. This ensures that only the code
path in which we start with an empty metadata log checks that the
metadata version is 7 or newer.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma
<ismael@juma.me.uk>, PoAn Yang <payang@apache.org>, Liu Zeyu
<zeyu.luke@gmail.com>, Alyssa Huang <ahuang@confluent.io>
The release script was pushing the RC tag off of a temporary branch that
was never merged back into the release branch. This meant that our RC
and release tags were detached from the rest of the repository.
This patch changes the release script to merge the RC tag back into the
release branch and pushes both the tag and the branch.
Reviewers: Luke Chen <showuon@gmail.com>
Under the `SHUTDOWN_APPLICATION` configuration in Kafka Streams, a tight
loop in the shutdown process can flood logs with repeated messages. This
PR introduces a check to ensure that the shutdown log is emitted only
once every 10 seconds, thereby preventing log flooding.
Reviewers: PoAn Yang <payang@apache.org>, Matthias J. Sax <matthias@confluent.io>
[KAFKA-18813](https://issues.apache.org/jira/browse/KAFKA-18813) added
`Topic:Describe` authorization of topics matching regex patterns to the
group coordinator since it was difficult to authorize these in the
broker when processing consumer heartbeats using the new protocol. But
group coordinator is started in `BrokerServer` before the authorizer is
created. And hence group coordinator doesn't have an authorizer and
never performs authorization. As a result, topics that are not
authorized for `Describe` may be assigned to consumers. This potentially
leaks information about topic existence, topic id and partition count to
users who are not authorized to describe a topic. This PR starts
authorizer earlier to ensure that authorization is performed by the
group coordinator. Also adds integration tests for verification.
Note that we still have a second issue when members have different
permissions. If regex is resolved by a member with permission to more
topics, unauthorized topics may be assigned to members with lower
permissions. In this case, we still return assignment containing topic
id and partitions to the member without `Topic:Describe` access. This is
not addressed by this PR, but an integration test that illustrates the
issue has been added so that we can verify when the issue is fixed.
Reviewers: David Jacot <david.jacot@gmail.com>
As of 3.9, Kafka allows disabling remote storage on a topic after it was
enabled. It allows subsequent enabling and disabling too.
However the documentation says otherwise and needs to be corrected.
Doc:
https://kafka.apache.org/39/documentation/#topicconfigs_remote.storage.enable
Reviewers: Luke Chen <showuon@gmail.com>, PoAn Yang <payang@apache.org>, Ken Huang <s7133700@gmail.com>
Call the StateRestoreListener#onBatchRestored with numRestored and not
the totalRestored when reprocessing state
See: https://issues.apache.org/jira/browse/KAFKA-18962
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias
Sax <mjsax@apache.org>
This PR fixes a potential issue where the `FetchResponse` returns
`divergingEndOffsets` with an older leader epoch. This can lead to
committed records being removed from the follower's log, potentially
causing data loss.
In detail:
`processFetchRequest` gets the requested leader epoch of partition data
by `topicPartition` and compares it with the leader epoch of the current
fetch state. If they don't match, the response is ignored.
Reviewers: Jun Rao <junrao@gmail.com>
For the KRaft implementation there is a race between the network thread,
which read bytes in the log segments, and the KRaft driver thread, which
truncates the log and appends records to the log. This race can cause
the network thread to send corrupted records or inconsistent records.
The corrupted records case is handle by catching and logging the
CorruptRecordException. The inconsistent records case is handle by only
appending record batches who's partition leader epoch is less than or
equal to the fetching replica's epoch and the epoch didn't change
between the request and response.
For the ISR implementation there is also a race between the network
thread and the replica fetcher thread, which truncates the log and
appends records to the log. This race can cause the network thread send
corrupted records or inconsistent records. The replica fetcher thread
already handles the corrupted record case. The inconsistent records case
is handle by only appending record batches who's partition leader epoch
is less than or equal to the leader epoch in the FETCH request.
Reviewers: Jun Rao <junrao@apache.org>, Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@apache.org>
`transform` and `through` are removed in 4.0. Since users cannot
reference them in 4.0 document, it's not good to keep using them as
example in `repartition` description.
Reviewers: Matthias J. Sax <matthias@confluent.io>
Fixes both KAFKA-16407 and KAFKA-16434.
Summary of existing issues:
- We are ignoring new left record when its previous FK value is null
- We do not unset foreign key join result when FK becomes null
Reviewers: Matthias J. Sax <matthias@confluent.io>
When a row in a FK-join left table is updated, we should send a "delete
subscription with no response" for the old FK to the right hand side, to
avoid getting two responses from the right hand side. Only the "new
subscription" for the new FK should request a response. If two responses
are requested, there is a race condition for which both responses could
be processed in the wrong order, leading to an incorrect join result.
This PR fixes the "delete subscription" case accordingly, to no request
a response.
Reviewers: Matthias J. Sax <matthias@confluent.io>
JIRA: KAFKA-18067
Fix producer client double-closing issue in Kafka Streams.
During StreamThread shutdown, TaskManager closes first, which closes the
producer client. Later, calling `unsubscribe` on the main consumer may
trigger the `onPartitionsLost` callback, attempting to reset
StreamsProducer when EOS is enabled. This causes an already closed
producer to be closed twice while the newly created producer is never
closed.
In detail:
This patch adds a flag to control the producer reset and has a new
method to change this flag, which is only invoked in
`ActiveTaskCreator#close`.
This would guarantee that the disable reset producer will only occur
when StreamThread shuts down.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
Backports f24945b519 to 4.0
Instead of reopening the transaction index, it cancels the RemoteFetchTask without interrupting it--avoiding to close the TransactionIndex channel.
This will lead to complete the execution of the remote fetch but ignoring the results. Given that this is considered a rare case, we could live with this. If it becomes a performance issue, it could be optimized.
Reviewers: Jun Rao <junrao@gmail.com>
Currently, each log.append() will add at most 1 index entry, even when
the appended data is larger than log.index.interval.bytes. One potential
issue is that if a follower restarts after being down for a long time,
it may fetch data much bigger than log.index.interval.bytes at a time.
This means that fewer index entries are created, which can increase the
fetch time from the consumers.
(cherry picked from commit e124d3975b)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Currently, the "Notable changes in 4.0.0" for the client is very confusing. We should remove it.
Reviewers: mingdaoy <mingdaoy@gmail.com>, Luke Chen <showuon@gmail.com>, Ken Huang <s7133700@gmail.com>, David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Fixes two issues:
- only commit TX if no revoked tasks need to be committed
- commit revoked tasks after punctuation triggered
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <sophie@responsive.dev>, Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>
Skip kraft.version when applying FeatureLevelRecord records. The kraft.version is stored as control records and not as metadata records. This solution has the benefits of removing from snapshots any FeatureLevelRecord for kraft.version that was incorrectly written to the log and allows ApiVersions to report the correct finalized kraft.version.
Reviewers: Colin P. McCabe <cmccabe@apache.org>