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>
The `AdminClient` adds a telemetry reporter to the metrics reporters
list in the constructor. The problem is that the reporter was already
added in the `createInternal` method. In the `createInternal` method
call, the `clientTelemetryReporter` is added to a
`List<MetricReporters>` which is passed to the `Metrics` object, will
get closed when `Metrics.close()` is called. But adding a reporter to
the reporters list in the constructor is not used by the `Metrics`
object and hence doesn't get closed, causing a memory leak.
All related tests pass after this change.
Reviewers: Apoorv Mittal <apoorvmittal10@apache.org>, Matthias J. Sax
<matthias@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>,
Jhen-Yung Hsu <jhenyunghsu@gmail.com>
This fixes librdkafka older than the recently released 2.11.0 with
Kerberos authentication and Apache Kafka 4.x.
Even though this is a bug in librdkafka, a key goal of KIP-896 is not to
break the popular client libraries listed in it. Adding back JoinGroup
v0 & v1 is a very small change and worth it from that perspective.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Log segment closure results in right sizing the segment on disk along
with the associated index files.
This is specially important for TimeIndexes where a failure to right
size may eventually cause log roll failures leading to under replication
and log cleaner failures.
This change uses `Utils.closeAll` which propagates exceptions, resulting
in an "unclean" shutdown. That would then cause the broker to attempt to
recover the log segment and the index on next startup, thereby avoiding
the failures described above.
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Jun Rao
<junrao@gmail.com>, Chia-Ping Tsai <chia7712@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>
### 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>
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>
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>
This patch is a first step towards resolving KAFKA-18046. Apache Kafka
4.0 ships with log4j2 so the issue raised in the ticket causing high CPU
usage on the fetch path due to LoggerFactory.getLogger() being called on
the handling of all fetch responses is not good. Hence, I propose to fix
that one by caching the Logger used by the `CompletedFetch` class.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Logging on a per-batch bases is very chatty, and should only be done at
TRACE level to avoid spamming DEBUG logs.
Reviewers: Justine Olshan <jolshan@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
In KRaft mode, custom KafkaPrincipalBuilder instances must implement KafkaPrincipalSerde. This PR updates all related documentation to highlight this requirement.
Reviewers: Ken Huang <s7133700@gmail.com>, David Jacot <djacot@confluent.io>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The purpose of this PR is to remove the `@InterfaceStability.Evolving` from classes that were created over a year ago.
Reviewers: Jun Rao <junrao@gmail.com>
KIP-966 adds strict min ISR rule, so this PR improves the docs of min.insync.replicas to include that change.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
… must check topic describe (#18989)
This patch filters out the topic describe unauthorized topics from the
ConsumerGroupHeartbeat and ConsumerGroupDescribe response.
In ConsumerGroupHeartbeat,
- if the request has `subscribedTopicNames` set, we directly check the
authz in `KafkaApis` and return a topic auth failure in the response if
any of the topics is denied.
- Otherwise, we check the authz only if a regex refresh is triggered and
we do it based on the acl of the consumer that triggered the refresh. If
any of the topic is denied, we filter it out from the resolved
subscription.
In ConsumerGroupDescribe, we check the authz of the coordinator
response. If any of the topic in the group is denied, we remove the
described info and add a topic auth failure to the described group.
(similar to the group auth failure)
(cherry picked from commit 36f19057e1)
Reviewers: David Jacot <djacot@confluent.io>, Lianet Magrans
<lmagrans@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>,
Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu <tjwu1217@gmail.com>,
TengYao Chi <kitingiao@gmail.com>
Cherry-picked
3603c8fe35
into 4.0.
This was a bug fix to address
https://issues.apache.org/jira/browse/KAFKA-18829.
Now, we will only move to IMPLICIT mode in `ShareConsumerImpl`, if there
were any records to be acknowledged, and if the next
`poll()`/`commitAsync()`/`commitSync()` was called.
Reviewers: Andrew Schofield <aschofield@confluent.io>
3.3.0 was the first KRaft release that was deemed production-ready and also
when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable
for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also
needs to be set to "3.3" or newer before upgrading).
Noteworthy changes:
1. `AlterPartition` no longer includes topic names, which makes it possible to
simplify `AlterParitionManager` logic.
2. Metadata versions older than `IBP_3_3_IV3` have been removed and
`IBP_3_3_IV3` is now the minimum version.
3. `MINIMUM_BOOTSTRAP_VERSION` has been removed.
4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`,
`isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and
`isInControlledShutdownStateSupported` - these are always `true` now.
Also removed related conditional code.
5. Removed default metadata version or metadata version fallbacks in
multiple places - we now fail-fast instead of potentially using an incorrect
metadata version.
6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord`
based on whether image is empty - this was a previously existing issue that
became more apparent after the changes in this PR.
7. Remove `ibp` parameter from `BootstrapDirectory`
8. A number of tests were not useful anymore and have been removed.
I will update the upgrade notes via a separate PR as there are a few things that
need changing and it would be easier to do so that way.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
Most of the changes are obvious clean-ups/fixes. A couple of noteworthy items:
1. Support for non LTS versions is clarified (we were incorrectly stating full support
for Java 23).
2. TLS version negotiation details are clarified.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Frequently updating the trust store can cause unexpected termination of the AsyncConsumer background thread.
1. To resolve this issue, reuse the same AdminClient instead of recreating it.
2. Add error logging when fail to initialize resources for the consumer network thread.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit ensures that the ClientQuotaCallback#updateClusterMetadata method is executed in KRaft mode. This method is triggered whenever a topic or cluster metadata change occurs. However, in KRaft mode, the current implementation of the updateClusterMetadata API is inefficient due to the requirement of creating a full Cluster object. To address this, a follow-up issue (KAFKA-18239) has been created to explore more efficient mechanisms for providing cluster information to the ClientQuotaCallback without incurring the overhead of a full Cluster object creation.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Return produce v0-v2 as supported versions in `ApiVersionsResponse`, but disable support
for it everywhere else.
Since clients pick the highest supported version by both client and broker during version
negotiation, this solves the problem with minimal tech debt (even though it's not ideal that
`ApiVersionsResponse` becomes inconsistent with the actual protocol support).
Add one test for the socket server handling (in `ProcessorTest`) and one test for the
client behavior (in `ProduceRequestTest`). Adjust a couple of api versions tests to verify
the new behavior.
Finally, include a few clean-ups in `ApiKeys`, `Protocol`, `ProduceRequest`,
`ProduceRequestTest` and `BrokerApiVersionsCommandTest`.
Reference to related librdkafka issue:
https://github.com/confluentinc/librdkafka/issues/4956
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
This change reduces fetch session cache evictions on the broker for AsyncKafkaConsumer by altering its logic to determine which partitions it includes in fetch requests.
Background
Consumer implementations fetch data from the cluster and temporarily buffer it in memory until the user next calls Consumer.poll(). When a fetch request is being generated, partitions that already have buffered data are not included in the fetch request.
The ClassicKafkaConsumer performs much of its fetch logic and network I/O in the application thread. On poll(), if there is any locally-buffered data, the ClassicKafkaConsumer does not fetch any new data and simply returns the buffered data to the user from poll().
On the other hand, the AsyncKafkaConsumer consumer splits its logic and network I/O between two threads, which results in a potential race condition during fetch. The AsyncKafkaConsumer also checks for buffered data on its application thread. If it finds there is none, it signals the background thread to create a fetch request. However, it's possible for the background thread to receive data from a previous fetch and buffer it before the fetch request logic starts. When that occurs, as the background thread creates a new fetch request, it skips any buffered data, which has the unintended result that those partitions get added to the fetch request's "to remove" set. This signals to the broker to remove those partitions from its internal cache.
This issue is technically possible in the ClassicKafkaConsumer too, since the heartbeat thread performs network I/O in addition to the application thread. However, because of the frequency at which the AsyncKafkaConsumer's background thread runs, it is ~100x more likely to happen.
Options
The core decision is: what should the background thread do if it is asked to create a fetch request and it discovers there's buffered data. There were multiple proposals to address this issue in the AsyncKafkaConsumer. Among them are:
The background thread should omit buffered partitions from the fetch request as before (this is the existing behavior)
The background thread should skip the fetch request generation entirely if there are any buffered partitions
The background thread should include buffered partitions in the fetch request, but use a small “max bytes” value
The background thread should skip fetching from the nodes that have buffered partitions
Option 4 won out. The change is localized to AbstractFetch where the basic idea is to skip fetch requests to a given node if that node is the leader for buffered data. By preventing a fetch request from being sent to that node, it won't have any "holes" where the buffered partitions should be.
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Jun Rao <junrao@gmail.com>
While testing, it was found that the not_enough_replicas error was super common and could be easily confused. Since we are already bumping the request, we can signify that the produce request may return this error and new clients can handle it
(Note, the java client should be able to handle this already as a retriable error, but other client libraries may need to implement this change)
Reviewers: Justine Olshan <jolshan@confluent.io>
Ensure we always return empty records (including cases where an error is returned).
We also remove `nullable` from `records` since it is effectively expected to be
non-null by a large percentage of clients in the wild.
This behavior regressed in fe56fc9 (KAFKA-18269). Empty records were
previously set via `FetchResponse.recordsOrFail(partitionData)` in the
now-removed `maybeConvertFetchedData` method.
Added an integration test that fails without this fix and also update many
tests to set `records` to `empty` instead of leaving them as `null`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
During testing, we identified that kafka-python (and aiokafka) relies on metadata request v0 and
hence we need to add these back to comply with the premise of KIP-896 - i.e. it should not
break the clients listed within it.
I reverted the changes from #18218 related to the removal of metadata versions 0-3.
I will submit a separate PR to undeprecate these API versions on the relevant 3.x branches.
kafka-python (and aiokafka) work correctly (produce & consume) with this change on
top of the 4.0 branch.
Reviewers: David Arthur <mumrah@gmail.com>
This patch reorganizes our test infrastructure into three Gradle modules:
":test-common:test-common-internal-api" is now a minimal dependency which exposes interfaces and annotations only. It has one project dependency on server-common to expose commonly used data classes (MetadataVersion, Feature, etc). Since this pulls in server-common, this module is Java 17+. It cannot be used by ":clients" or other Java 11 modules.
":test-common:test-common-util" includes the auto-quarantined JUnit extension. The @Flaky annotation has been moved here. Since this module has no project dependencies, we can add it to the Java 11 list so that ":clients" and others can utilize the @Flaky annotation
":test-common:test-common-runtime" now includes all of the test infrastructure code (TestKitNodes, etc). This module carries heavy dependencies (core, etc) and so it should not normally be included as a compile-time dependency.
In addition to this reorganization, this patch leverages JUnit SPI service discovery so that modules can utilize the integration test framework without depending on ":core". This will allow us to start moving integration tests out of core and into the appropriate sub-module. This is done by adding ":test-common:test-common-runtime" as a testRuntimeOnly dependency rather than as a testImplementation dependency. A trivial example was added to QuorumControllerTest to illustrate this.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
We should provide the same informative error message for both timeout
cases.
Reviewers: Kirk True <ktrue@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Kafka 4.0 will remove support for zk mode and will require conversion to kraft
before upgrading to 4.0. The minimum kraft version is 3.0 (aka 3.0-IV1).
This provides an opportunity to remove exclusively server side protocols versions
that only exist to allow direct upgrades from versions older than 3.0 or that are
used only by zk mode.
Since KRaft became production ready in 3.3, we should consider setting the
baseline to 3.3. But that requires more discussion and it can be done via a
separate change (KAFKA-18601).
Protocol changes:
* Remove RequestHeader v0 (only used by ControlledShutdown v0)
* Remove WriteTxnMarkers v0
* Remove all versions of ControlledShutdown, LeaderAndIsr, StopReplica, UpdateMetadata
In order to remove all versions safely, extend generator to support setting
"versions" to "none". In this case, we no longer generate the `*Data` classes,
but we still reserve the id for the relevant protocol api (so it doesn't get
accidentally used for something else). The protocol documentation is correct
after these changes.
We kept a simplified version of `LeaderAndIsr{Request|Response}` because
it's used by many tests that are still relevant in kraft mode. Once
KAFKA-18486 is done, it may be possible to remove it (I left a comment on
the ticket). Similarly, KAFKA-18487 may make it possible to remove
the introduced `StopReplicaPartitionState` (left a comment on that ticket too).
There are a number of places that were adjusted to include an
`ApiKeys.hasValidVersion` check.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
A minor refactoring just before merging #18295 introduced a regression and no test failed. Throw the correct exception and add test to verify it. Also refactor the code slightly to make that possible.
Thanks to Chia-Ping for catching the issue.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR implements the second part of KIP-996 and KAFKA-16164 (tasks KAFKA-16607, KAFKA-17642, KAFKA-17643, KAFKA-17675) which encompass the response handling of PreVotes, addition of new ProspectiveState, update to metrics, and addition of Raft simulation tests.
Voters now transition to ProspectiveState first before CandidateState to prevent unnecessary epoch bumps. Voters in ProspectiveState send PreVotes requests which are Vote requests with PreVote set to true.
Follower grants PreVotes if it has not yet fetched successfully from leader. Leader denies all PreVotes. Unattached, Prospective, Candidate, and Resigned will grant PreVotes if the requesting replica's log is at least as long as theirs. Granted PreVotes are not persisted like standard votes. It is possible for a voter to grant several PreVotes in the same epoch.
The only state which is allowed to transition directly to CandidateState is ProspectiveState. This happens on reception of majority of granted PreVotes or if at least one voter doesn't support PreVote requests.
Prospective will transition to Follower after election loss/timeout if it was already aware of last known leader and the leader's endpoint, or at any point if it discovers the leader.
Prospective will transition to Unattached after election loss/timeout if it does not know the leader endpoints.
After electionTimeout, Resigned now always transitions to Unattached and increases the epoch.
Prospective grants standard votes if it has not already granted a standard vote (no votedKey), has no leaderId, and the recipient's log is current enough
Candidate no longer backs off after election timeout. Candidate still backs off after election loss.
Reviewers: José Armando García Sancio <jsancio@apache.org>