This patch mainly includes two improvements:
1. Update currentFetch when `pollForFetches()` throws an exception.
2. Add an override `KafkaShareConsumer.acknowledge(String topic, int
partition, long offset, AcknowledgeType type)` .
Reviewers: Andrew Schofield <aschofield@confluent.io>
Another refactor PR to move in-flight batch and state out of
SharePartition. This PR concludes the refactoring and subsequent PRs for
this ticket will involve code cleanups and better lock handling. However
the intent is to keep PRs small so they can be reviewed easily.
Reviewers: Andrew Schofield <aschofield@confluent.io>
While working on KAFKA-19476, I realized that we need to refactor
SharePartition for read/write lock handling. I have started some work in
the area. For the initial PR, I have moved AcquisitionLockTimeout class
outside of SharePartition.
Reviewers: Andrew Schofield <aschofield@confluent.io>
Improve the error handling while building the remote-log-auxiliary state
when a follower node with an empty disk begin to synchronise with the
leader. If the topic has remote storage enabled, then the
ReplicaFetcherThread attempt to build the remote-log-auxiliary state.
Note that the remote-log-auxiliary state gets invoked only when the
leader-log-start-offset is non-zero and leader-log-start-offset is not
equal to leader-local-log-start-offset.
When the LeaderAndISR request is received, then the
ReplicaManager#becomeLeaderOrFollower invokes 'makeFollowers' initially,
followed by the RemoteLogManager#onLeadershipChange call. As a result,
when ReplicaFetcherThread initiates the
RemoteLogManager#fetchRemoteLogSegmentMetadata, the partition may not
have been initialized at that time and throws retriable exception.
Introduced RetriableRemoteStorageException to gracefully handle the
error.
After the patch:
```
[2025-07-19 19:28:20,934] INFO [ReplicaFetcher replicaId=3, leaderId=1,
fetcherId=0] Could not build remote log auxiliary state for orange-1 due
to error: RemoteLogManager is not ready for partition: orange-1
(kafka.server.ReplicaFetcherThread)
[2025-07-19 19:28:20,934] INFO [ReplicaFetcher replicaId=3, leaderId=2,
fetcherId=0] Could not build remote log auxiliary state for orange-0 due
to error: RemoteLogManager is not ready for partition: orange-0
(kafka.server.ReplicaFetcherThread)
```
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
Changes: Rename `waitForTopic` to `waitTopicCreation` for better clarity
Reasons: To align with `waitTopicDeletion` Reference:
https://github.com/apache/kafka/pull/20108/files#r2221659660
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<frankvicky@apache.org>
This feature adds maintenance burden and potential security concerns
while providing no apparent value to the Kafka community. See
[KIP-1193](https://cwiki.apache.org/confluence/x/dAxJFg) for more
details.
Reviewers: TengYao Chi <frankvicky@apache.org>, Ken Huang
<s7133700@gmail.com>
---------
Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
The PR refactors the findNextFetchOffset variable from AtomicBoolean to
boolean itself as the access is always done while holding a lock. This
also improves handling of `writeShareGroupState` method response where
now complete lock is not required, rather on sub-section.
Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield
<aschofield@confluent.io>
Add its for `Admin.deleteShareGroupOffsets`,
`Admin.alterShareGroupOffsets` and `Admin.listShareGroupOffsets` to
`PlaintextAdminIntegrationTest`.
Reviewers: Andrew Schofield <aschofield@confluent.io>
This patch adds an API level integration test for the producer epoch
verification when processing transactional offset commit and end txn
markers.
Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<kitingiao@gmail.com>, Sean Quah <squah@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
Improve the error message in the kafka-storage.sh when an incorrect
release-version is given. Specifically, following the behavior of
kafka-feature.sh, when an incorrect release-version is entered, it
returns the currently supported versions to the user.
Reviewers: TengYao Chi <frankvicky@apache.org>, Yung
<yungyung7654321@gmail.com>
The MetadataImage has a lot of stuff in it and it gets passed around in
many places in the new GroupCoordinator. This makes it difficult to
understand what metadata the group coordinator actually relies on and
makes it too easy to use metadata in ways it wasn't meant to be used.
This change encapsulate the MetadataImage in an interface
(`CoordinatorMetadataImage`) that indicates and controls what metadata
the group coordinator actually uses. Now it is much easier at a glance
to see what dependencies the GroupCoordinator has on the metadata. Also,
now we have a level of indirection that allows more flexibility in how
the GroupCoordinator is provided the metadata it needs.
`ReplicaManager#alterReplicaLogDirs` does not resume log cleaner while
handling an `AlterReplicaLogDirs` request for a topic partition which
already has an `AlterReplicaLogDirs` in progress, leading to a resource
leak where the cleaning for topic partitions remains paused even after
the log directory has been altered.
This change ensures we invoke `LogManager#resumeCleaning` if the future
replica directory has changed.
Reviewers: Jun Rao <junrao@gmail.com>
Removing the isEligibleLeaderReplicasV1Enabled to let ELR be enabled if
MV is at least 4.1IV1. Also bump the Latest Prod MV to 4.1IV1
Reviewers: Paolo Patierno <ppatierno@live.com>, Jun Rao <junrao@gmail.com>
### Summary of Changes
- Rewrote both `CoordinatorLoaderImpl` and `CoordinatorLoaderImplTest`
in Java, replacing their original Scala implementations.
- Removed the direct dependency on `ReplicaManager` and replaced it with
functional interfaces for `partitionLogSupplier` and
`partitionLogEndOffsetSupplier`
- Preserved original logic and test coverage during migration.
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>,
TengYao Chi <frankvicky@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
The mocked value for `UnifiedLog#topicId` was incorrectly set up which
caused test failure.
Reviewers: Luke Chen <showuon@gmail.com>, PoAn Yang <payang@apache.org>, Satish Duggana <satishd@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
### Background
As part of KIP-932 implementation, ShareFetch requests need to properly
integrate with Kafka's quota system. This requires that ShareFetch
requests extract and pass the correct session information (Principal,
client address, client ID) to quota managers, ensuring consistent quota
enforcement between ShareFetch and traditional Fetch requests.
### Changes
This PR adds `testHandleShareFetchRequestQuotaTagsVerification()`,
`testHandleShareAcknowledgeRequestQuotaTagsVerification` and
`testHandleShareFetchWithAcknowledgementQuotaTagsVerification` to
`KafkaApisTest`, which provides verification of quota tag extraction and
session handling for ShareFetch and ShareAcknowledge requests.
- Ensures ShareFetch/ShareAck requests are properly constructed with
the correct client ID, principal, client address, and API key
- Verifies the request context contains the expected session
information
- Uses `ArgumentCaptor` to capture the exact `Session` and
`RequestChannel.Request` objects passed to quota managers
- Verifies both `quotas.fetch.maybeRecordAndGetThrottleTimeMs()` and
`quotas.request.maybeRecordAndGetThrottleTimeMs()` are called with
correct parameters as and when needed.
- Validates that the captured `RequestChannel.Request` object
maintains the correct request context information
- Ensures the client ID passed to quota managers matches the
test-defined value
- Verifies that in case of Acks being piggybacked on the fetch
requests, the quotas are applied only once and not twice.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
### Summary
Extends RequestQuotaTest to include ShareFetch API quota testing,
ensuring compliance with KIP-932.
### Key Changes
- New test: testShareFetchUsesSameFetchSensor() - Verifies ShareFetch
and Fetch use the same FETCH quota sensor
- New test:
testResponseThrottleTimeWhenBothShareFetchAndRequestQuotasViolated() -
Tests ShareFetch throttling behaviour
- Request builder: Added ApiKeys.SHARE_FETCH case with proper ShareFetch
request construction
- Some minor cleanup wrt use of Collections
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
This PR adds the following metrics for each of the supported production
features (`metadata.version`, `kraft.version`, `transaction.version`,
etc.):
`kafka.server:type=MetadataLoader,name=FinalizedLevel,featureName=X`
`kafka.server:type=node-metrics,name=maximum-supported-level,feature-name=X`
`kafka.server:type=node-metrics,name=minimum-supported-level,feature-name=X`
Reviewers: Josep Prat <josep.prat@aiven.io>, PoAn Yang
<payang@apache.org>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Lan Ding
<isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
The `testHWCheckpointWithFailuresMultipleLogSegments` test in
`LogRecoveryTest` was failing intermittently due to a race condition
during its failure simulation.
In successful runs, the follower broker would restart and rejoin the
In-Sync Replica (ISR) set before the old leader's failure was fully
processed. This allowed for a clean and timely leader election to the
now in-sync follower.
However, in the failing runs, the follower did not rejoin the ISR before
the leader election was triggered. With no replicas in the ISR and
unclean leader election disabled by default for the test, the controller
correctly refused to elect a new leader, causing the test to time out.
This commit fixes the flakiness by overriding the controller
configuration for this test to explicitly enable unclean leader
election. This allows the out-of-sync replica to be promoted to leader,
making the test deterministic and stable.
Reviewers: Jun Rao <junrao@gmail.com>
This PR enables reading remote storage for multiple partitions in one
fetchRequest. The main changes are:
1. In `DelayedRemoteFetch`, we accept multiple remoteFetchTasks and
other metadata now.
2. In `DelayedRemoteFetch`, we'll wait until all remoteFetch done,
either succeeded or failed.
3. In `ReplicaManager#fetchMessage`, we'll create one
`DelayedRemoteFetch` and pass multiple remoteFetch metadata to it, and
watch all of them.
4. Added tests
Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Federico Valeri <fedevaleri@gmail.com>, Satish Duggana <satishd@apache.org>
Fixup PR Labels / fixup-pr-labels (needs-attention) (push) Has been cancelledDetails
Fixup PR Labels / fixup-pr-labels (triage) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.7.2) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.8.1) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.9.1) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (4.0.0) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (latest) (push) Has been cancelledDetails
Fixup PR Labels / needs-attention (push) Has been cancelledDetails
Flaky Test Report / Flaky Test Report (push) Has been cancelledDetails
Moving rollback out of lock, if persister returns a completed future for
write state then same data-plane-request-handler thread should not call
purgatory safeTryAndComplete while holding SharePartition's write lock.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit
<adixit@confluent.io>
- Fix testUncommittedDataNotConsumedFrequentSegmentRolls() and
testUncommittedDataNotConsumed(), which call createLog() but never close
the log when the tests complete.
- Move LogConcurrencyTest to the Storage module and rewrite it in Java.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
#5608 introduced a regression where the check for `targetOffset <
log.highWatermark`
to emit a `WARN` log was made incorrectly after truncating the log.
This change moves the check for `targetOffset < log.highWatermark` to
`UnifiedLog#truncateTo` and ensures we emit a `WARN` log on truncation
below the replica's HWM by both the `ReplicaFetcherThread` and
`ReplicaAlterLogDirsThread`
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Creates GetReplicaLogInfoRequest and GetReplicaLogInfoResponse RPCs
Information returned by these brokers will be used to aid
unclean-recovery by selecting longest logs.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Calvin Liu <caliu@confluent.io>, Colin P. McCabe <cmccabe@apache.org>, TaiJuWu <tjwu1217@gmail.com>
PlaintextConsumerTest should extend AbstractConsumerTest instead
BaseConsumerTest. Otherwise, those tests will be executed on both
`clients-integration-tests` and `core` (see
https://github.com/apache/kafka/pull/20081/files#r2190749592).
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Use Java to rewrite PlaintextConsumerTest by new test infra and move it
to client-integration-tests module.
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Migrate ControllerMutationQuotaManager to Java implementation and move
to server module, including ClientQuotaManager and associated files.
Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
While testing the code in https://github.com/apache/kafka/pull/19820, it
became clear that the error handling problems were due to the underlying
Admin API. This PR fixes the error handling for top-level errors in the
AlterShareGroupOffsets RPC.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Lan Ding
<isDing_L@163.com>, TaiJuWu <tjwu1217@gmail.com>
Estimate the fetch size for remote fetch to avoid to exceed the
`fetch.max.bytes` config. We don't want to query the remoteLogMetadata
during API handling, thus we assume the remote fetch can get
`max.partition.fetch.bytes` size. Tests added.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
### About
`nextFetchOffset` function in `SharePartition` updates the fetch offsets
without considering batches/offsets which might be undergoing state
transition. This can cause problems in updating to the right fetch
offset.
### Testing
The new code added has been tested with the help of unit tests.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
issue: https://github.com/apache/kafka/pull/19687/files#r2094574178
Why:
- To improve performance by avoiding redundant temporary collections and
repeated method calls.
- To make the utility more flexible for inputs from both Java and Scala.
What:
- Refactored `createResponseConfig` in `ConfigHelper.scala` by
overloading the method to accept both Java maps and `AbstractConfig`.
- Extracted helper functions to `ConfigHelperUtils` in the server
module.
Reviewers: Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
- remove `threadNamePrefix` from `ReplicaManager` constructor
- update `BrokerServer` to use updated constructor
- remove `threadNamePrefix` from `ReplicaFetcherManager`
Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<frankvicky@apache.org>
The PR do following:
1. Remove ReplicaManager#becomeLeaderOrFollower.
2. Remove `LeaderAndIsrRequest` and `LeaderAndIsrResponse`
3. Migrate `LeaderAndIsrRequest.PartitionState` to server-common module
and change to `PartitionState`
4. Remove `ControllerEpoch` from PartitionState
5. Remove `isShuttingDown` from BrokerServer and ReplicaManager
Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
This PR does the following:
1) Rewrites consumerBounceTest in Java.
2) Moves the test to clients-integration-test.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Fixup PR Labels / fixup-pr-labels (needs-attention) (push) Has been cancelledDetails
Fixup PR Labels / fixup-pr-labels (triage) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.7.2) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.8.1) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (3.9.1) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (4.0.0) (push) Has been cancelledDetails
Docker Image CVE Scanner / scan_jvm (latest) (push) Has been cancelledDetails
Fixup PR Labels / needs-attention (push) Has been cancelledDetails
Refactor the code related to SocketServer SocketServer is an internal
class, and normally the integration tests should not use it directly.
[KAFKA-19239](https://issues.apache.org/jira/browse/KAFKA-19239) will
add a new helper to expose the bound ports, and so the tests that need
to send raw request can leverage it without accessing the SocketServer.
Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Ensure the config.providers configuration is documented for all
components supporting it
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Greg Harris
<gharris1727@gmail.com>, Matthias J. Sax <mjsax@apache.org>
In the stress testing it was noticed that on acquisition lock timeout,
some offsets were not found in the cache. The cache can be tried to be
updated in different acknowledgement calls hence if there is an ongoing
transition which is not yet finished but another parallel
acknowledgement triggers the cache update then the cache can be updated
incorrectly, while first transition is not yet finished.
Though the cache update happens for Archived and Acknowldeged records
hence this issue or existing implementation should not hamper the queues
functionality. But it might update the cache early when persister call
might fail or this issue triggers error logs with offset not found in
cache when acquisition lock timeouts (in some scenarios).
Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield
<aschofield@confluent.io>
The OffsetFetch API does not support top level errors in version 1.
Hence, the top level error must be returned at the partition level.
Side note: It is a tad annoying that we create error response in
multiple places (e.g. KafkaApis, Group CoordinatorService). There were a
reason for this but I cannot remember.
Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Sean Quah <squah@confluent.io>, Ken Huang <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
Now that Kafka supports Java 17, this PR cleans up the
ShareFetchAcknowledgeRequestTest.
The changes mostly include:
- Collections.singletonList() is replaced with List.of()
- Get rid of all asJava conversions
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
In `kafka.server.ClientQuotaManager` class, `quotaTypesEnabled` is not updated when a quota is removed via `removeQuota` method in `DefaultQuotaCallback` class. This field is set when quotas are added in `updateQuota` but it's never changed or cleared. So in case all the quotas have been removed dynamically, the system may incorrectly assume the quotas are active, which leads to unnecessary metric creation or updates until the broker is restarted.
Reviewers: Jonah Hooper <jhooper@confluent.io>, Hailey Ni <hni@confluent.io>, Alyssa Huang <ahuang@confluent.io>, David Jacot <djacot@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
continues the migration effort for KAFKA-18486 by replacing usage of the
deprecated `becomeLeaderOrFollower` API with `applyDelta` in several
test cases.
#### Updated tests:
- `testInconsistentIdReturnsError`
- `testMaybeAddLogDirFetchers`
- `testMaybeAddLogDirFetchersPausingCleaning`
- `testSuccessfulBuildRemoteLogAuxStateMetrics`
- `testVerificationForTransactionalPartitionsOnly`
- `testBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate`
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
<tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
ConsumerGroupDescribe with an empty group id returns a response containing `null` groupId in a non-nullable field. Since the response cannot be serialized, this results in UNKNOWN_SERVER_ERROR being returned to the client. This PR sets the group id in the response to an empty string instead and adds request tests for empty group id.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
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>
To maintain code consistency, `MetadataVersion#fromVersionString` uses
`latestTesting()` as the latest version. Therefore, in the tools, we
also need to maintain consistency by updating the outer logging to use
`latestTesting()`.
See the discussion:
https://github.com/apache/kafka/pull/18845#discussion_r1950706791
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Correct the error when SharePartition is fenced.
Reviewers: Abhinav Dixit <adixit@confluent.io>, Sushant Mahajan
<smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
* The `SharePartition` class wraps the errors received from
`PersisterStateManager` to be sent to the client.
* In this PR, we are categorizing the errors a bit better.
* Some exception messages in `PersisterStateManager` have been updated
to show the share partition key.
* Tests have been updated wherever needed.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal
<apoorvmittal10@gmail.com>
Description
* Replace `org.apache.kafka.common.test.TestUtils` with
`org.apache.kafka.test.TestUtils` in outer package modules to
standardize test utility usage
* Move `waitUntilLeaderIsElectedOrChangedWithAdmin` method from
`org.apache.kafka.test.TestUtils` to `ClusterInstance` and refactor for
better code organization
* Add `org.apache.kafka.test.TestUtils` dependency to
`transaction-coordinator` import control
Reviewers: PoAn Yang [payang@apache.org](mailto:payang@apache.org), Ken
Huang [s7133700@gmail.com](mailto:s7133700@gmail.com), Ken Huang
[s7133700@gmail.com](mailto:s7133700@gmail.com), Chia-Ping Tsai
[chia7712@gmail.com](mailto:chia7712@gmail.com)
In KRaft, custom KafkaPrincipalBuilder instances must implement
KafkaPrincipalSerde to support the forward mechanism. Currently, this
requirement is not enforced and relies on the developer's attention.
With this patch, we can prevent incorrect implementations at compile
time.
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>