Commit Graph

16106 Commits

Author SHA1 Message Date
Apoorv Mittal d350f603a4
KAFKA-18265: Move inflight batch and state classes from SharePartition (2/N) (#20230)
CI / build (push) Waiting to run Details
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>
2025-07-23 23:01:23 +01:00
Apoorv Mittal a663ce3f45
KAFKA-18265: Move acquisition lock classes from share partition (1/N) (#20227)
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>
2025-07-23 20:21:42 +01:00
Kamal Chandraprakash 93adaea599
KAFKA-19523: Gracefully handle error while building remoteLogAuxState (#20201)
CI / build (push) Waiting to run Details
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>
2025-07-23 19:29:31 +05:30
Chang-Chi Hsu 0086f24101
MINOR: Declare the inner RocksDBDualCFRangeIterator class as static (#20220)
Make inner classes static.

from: https://github.com/apache/kafka/pull/20182#issuecomment-3102893453

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-07-23 21:37:48 +08:00
Kamal Chandraprakash 16c079ed23
KAFKA-19525: Refactor TopicBasedRLMM implementation to remove unused code (#20204)
CI / build (push) Waiting to run Details
- startConsumerThread is always true so removed the variable.
- Replaced the repetitive lock handling logic with
`withReadLockAndEnsureInitialized` to reduce duplication and improve
readability.
- Consolidated the logic in `initializeResources` and. simplified method
arguments to better encapsulate configuration.
- Extracted common code and reduced the usage of global variables.
- Named the variables properly.

Tests:
- Existing UTs since this patch refactored the code.

Reviewers: PoAn Yang <payang@apache.org>
2025-07-23 12:19:13 +05:30
Sanskar Jhajharia f1e9aa1c65
MINOR: Fix flaky tests in Tools modules (#20225)
### Problem
The
`ShareGroupCommandTest.testDeleteShareGroupOffsetsArgsWithoutTopic()`,
`ShareGroupCommandTest.testDeleteShareGroupOffsetsArgsWithoutGroup()`,
`ResetStreamsGroupOffsetTest.testResetOffsetsWithoutGroupOption()`,
`DeleteStreamsGroupTest.testDeleteWithoutGroupOption()`,
`DescribeStreamsGroupTest.testDescribeWithoutGroupOption()` tests were
flaky due to a dependency on Set iteration order in error message
generation.

### Root Cause
The cleanup [commit](https://github.com/apache/kafka/pull/20091) that
replaced `new HashSet<>(Arrays.asList(...))` with `Set.of(...)` in
ShareGroupCommandOptions and StreamsGroupCommandOptions changed the
iteration characteristics of collections used for error message
generation:

This produces different orders like `[topic], [group]` vs `[group],
[topic]`, but the tests expected a specific order, causing intermittent
failures.

### Solution
Fix the root cause by ensuring deterministic error message generation
through alphabetical sorting of option names.

Reviewers: ShivsundarR <shr@confluent.io>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-07-23 14:40:18 +08:00
Ritika Reddy c7e4ff01cd
KAFKA-19272: Add initPid Response handling when keepPrepared is set to true (KIP-939) (#20039)
CI / build (push) Waiting to run Details
When initPid(keepPrepared = true) is called after a client crashes,
several situations should be considered.

When there's an ongoing transaction, we can transition it to the newly
added PREPARED_TRANSACTION state. However, what if there's no ongoing
transaction?

Another scenario could be:

- Issued a commit, to commit prepared
- The commit succeeded on the TC, but the client crashed
- Client restarted with keepPreparedTxn=true (because it doesn't know if
the commit succeeded or not and needs to keep retrying the commit until
it's successful)
- Issued a commit, but the transaction is not ongoing, because it's
committed

**Solution:**
This is a perfectly valid scenario as the external transaction
coordinator for the 2PC transaction will keep committing participants,
and the participants need to eventually return success (that's a
guarantee for a prepared transaction).
_Rejected Alt 1_ -> Return an InvalidTxnStateException : Returning an
error would break the above scenario.
_Rejected Alt 2_ -> Then the next thought is that we should somehow
validate if the state is expected, but we don't have data to validate
the result against.

**Final Solution:**  Just returning the success and transitioning to
READY is the proper handling of this condition.

Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
 <alivshits@confluent.io>
2025-07-22 15:03:49 -07:00
Evanston Zhou 1cef5325ad
KAFKA-19213 Client ignores default properties (#20134)
https://issues.apache.org/jira/browse/KAFKA-19213

Fixes a bug where creating a producer/consumer using a `Properties`
object created using the `Properties(Properties defaults)` constructor
will ignore the default properties.

Reviewers: Kirk True <kirk@kirktrue.pro>, TaiJuWu <tjwu1217@gmail.com>,
 Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-07-23 02:16:58 +08:00
Lan Ding 7738db9b2d
KAFKA-19533: Correct the docs of 'messages' for `kafka-consumer-perf-test.sh` and `kafka-share-consumer-perf-test.sh (#20217)
The descrption "REQUIRED: The number of messages to send or consume" is
not correct, since those tools do NOT send any records.

Reviewers: TengYao Chi <frankvicky@apache.org>
2025-07-22 22:37:49 +08:00
Sanskar Jhajharia 50b748b024
MINOR: Cleanup Connect Module (2/n) (#19871)
Now that Kafka support Java 17, this PR makes some changes in connect
module. The changes in this PR are limited to only some files. A future
PR(s) shall follow.
The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

Modules target: test-plugins, transforms

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-07-22 22:23:54 +08:00
Chang-Chi Hsu 8a5549ca9b
MINOR: Rename waitForTopic to waitTopicCreation (#20216)
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>
2025-07-22 21:02:57 +08:00
Federico Valeri f5fcc4188f
KAFKA-19503: Deprecate MX4j support (#20208)
CI / build (push) Waiting to run Details
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>
2025-07-22 20:36:24 +08:00
Sanskar Jhajharia 5cf6a9d80d
MINOR: Cleanup Tools Module (1/n) (#20091)
Now that Kafka support Java 17, this PR makes some changes in tools
module. The changes in this PR are limited to only some files. A future
PR(s) shall follow.
The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

Sub modules targeted: tools/src/main

Reviewers: Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-07-22 18:14:25 +08:00
Evgeniy Kuvardin dfd996e51e
KAFKA-18336: Improve jmh tests on ACL in AuthorizerBenchmark and StandardAuthorizerUpdateBenchmark (#18293)
1. JMH test should return value against return void (compiler can
eliminate returned value and benchmark would be incorrect).
2. Also move constant variable from method to class, to prevent JIT to
unfold.
3. Increase warm up iterations

Reviewers: Lucas Brutschy <lucasbru@apache.org>
2025-07-22 11:07:07 +02:00
Jinhe Zhang ed22a8b6cb
MINOR: Fix StreamThreadTest (#20210)
There is a typo in the unit test, it calls
`runOnceWithoutProcessingThreads` while it should call
`runOnceWithProcessingThreads` instead.

Reviewers: Lucas Brutschy <lucasbru@apache.org>
2025-07-22 11:04:23 +02:00
Sushant Mahajan 6ef675d08b
MINOR: Change log level for cold snapshot log. (#20209)
CI / build (push) Waiting to run Details
* We INFO log a message, if a share partition could be cold snapshotted.
* However, this may create noise if we have highly partitioned topic
backing the share partition. This will be further exacerbated by
multiple share groups using that topic.
* To reduce log pollution, this PR changes the level to DEBUG.

Reviewers: ShivsundarR <shr@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-07-21 16:13:37 +01:00
Gasparina Damien cdc2d957ed
KAFKA-16505: Adding dead letter queue in Kafka Streams (#17942)
Implements KIP-1034 to add support of Dead Letter
Queue in Kafka Streams. 

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Bruno Cadonna
 <cadonna@apache.org>
Co-authored-by: Sebastien Viale <sebastien.viale@michelin.com>
2025-07-21 15:54:40 +02:00
Apoorv Mittal f52f2b99e5
KAFKA-19476: Removing AtomicBoolean for findNextFetchOfffset (1/N) (#20207)
CI / build (push) Waiting to run Details
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>
2025-07-21 13:12:13 +01:00
xijiu f188a31124
KAFKA-19500: `kafka-consumer-groups.sh` should fail quickly if the partition leader is unavailable (#20168)
1. Add check leader missing logic in method
`ConsumerGroupCommand.ConsumerGroupService#prepareOffsetsToReset` in
order to fail quickly
2. Add some tests

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Lan Ding <isDing_L@163.com>,
 Ken Huang <s7133700@gmail.com>, Andrew Schofield
 <aschofield@confluent.io>
2025-07-21 09:25:40 +01:00
Lan Ding ef07b5fad1
KAFKA-19461: Add share group admin integration tests to PlaintextAdminIntegrationTest (#20103)
Add its for `Admin.deleteShareGroupOffsets`,
`Admin.alterShareGroupOffsets` and `Admin.listShareGroupOffsets`  to
`PlaintextAdminIntegrationTest`.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-07-21 09:08:26 +01:00
Logan Zhu 24d03a18ef
KAFKA-19517: Include control records in LoadSummary#numRecords (#20206)
## Summary
jira: https://issues.apache.org/jira/browse/KAFKA-19517
Ensure `LoadSummary#numRecords` counts all records, including control
batches, to maintain consistency with numBytes.

## Test
`testLoading` now verifies `numRecords`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
 <frankvicky@apache.org>
2025-07-21 15:12:18 +08:00
Dongnuo Lyu 50598191dc
MINOR: Add tests on TxnOffsetCommit and EndTxnMarker protection against invalid producer epoch when TV2 is used (#20024)
CI / build (push) Waiting to run Details
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>
2025-07-21 06:34:29 +08:00
Dmitry Werner 634e99e9ab
MINOR: Cleanup metadata module (#20115)
- Removed unused methods and arguments;
- Used enhanced switch and functional style expression for Optional;
- Fixed IDEA code inspection warnings.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-07-21 04:51:09 +08:00
Lan Ding 9a2f202a1e
MINOR: Move ClientQuotasRequestTest to server module (#20053)
CI / build (push) Waiting to run Details
1. Move ClientQuotasRequestTest to server module.
2. Rewrite ClientQuotasRequestTest in Java.

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-07-20 23:14:55 +08:00
Calvin Liu c162d2eb14
KAFKA-19522: avoid electing fenced lastKnownLeader (#20200)
CI / build (push) Waiting to run Details
This patch fixes the bug that allows the last known leader to be elected as a partition leader while still in a fenced state, before the next heartbeat removes the fence.
https://issues.apache.org/jira/browse/KAFKA-19522

Reviewers: Jun Rao <junrao@gmail.com>, TengYao Chi
<frankvicky@apache.org>
2025-07-20 16:16:37 +08:00
Lan Ding 908049fccc
MINOR: add a test for Protocol (#20169)
CI / build (push) Waiting to run Details
see https://github.com/apache/kafka/pull/19769#issuecomment-3065869429
This patch adds a test to `ProtocolTest` to ensure the Protocol page displays the correct API version range.

Reviewers: Yung <yungyung7654321@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Gaurav Narula <gaurav_narula2@apple.com>, Ken
Huang <s7133700@gmail.com>, Jimmy Wang <wangzhiwang@qq.com>
2025-07-19 19:31:25 +08:00
Logan Zhu 1b351ad6e2
MINOR: Remove unnecessary dependencies from coordinator-common (follow up to pr#20089) (#20194)
CI / build (push) Waiting to run Details
This PR removes the dependencies on `core` and `scala-library` from the
`coordinator-common` module, as a follow-up to
https://github.com/apache/kafka/pull/20089.

These dependencies have been removed from tests, and the previously
added import-control relaxations have been reverted accordingly.

Reviewers: TengYao Chi <frankvicky@apache.org>, Ken Huang
<s7133700@gmail.com>
2025-07-19 19:08:33 +08:00
Dmitry Werner 93a88a940b
KAFKA-19520 Bump Commons-Lang for CVE-2025-48924 (#20196)
Bump Commons-Lang for CVE-2025-48924.

Reviewers: Luke Chen <showuon@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2025-07-19 15:05:50 +08:00
Jinhe Zhang 38e3359446
KAFKA-19511: Fix flaky test HandlingSourceTopicDeletionIntegrationTest.shouldThrowErrorAfterSourceTopicDeleted (#20187)
CI / build (push) Waiting to run Details
Temporarily fix it by disable the new protocol, will take a deeper look
at it in the consumer protocol.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-07-18 21:51:45 +02:00
majialong 9b542b6ea2
MINOR: Correct RemoteLogManager.getLeaderEpochEntries comment (#20181)
CI / build (push) Waiting to run Details
The comment on the RemoteLogManager.getLeaderEpochEntries method has a
small error description,it should be start(inclusive)and end(exclusive).

Reviewers: Ken Huang <s7133700@gmail.com>, Lan Ding <isDing_L@163.com>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-07-19 00:04:03 +08:00
Lan Ding 9572d19c59
KAFKA-19509: Improve error message when release version is wrong (#20185)
CI / build (push) Waiting to run Details
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>
2025-07-18 11:39:55 +08:00
Elizabeth Bennett f81853ca88
KAFKA-19441: encapsulate MetadataImage in GroupCoordinator/ShareCoordinator (#20061)
CI / build (push) Waiting to run Details
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.
2025-07-18 08:16:54 +08:00
Dmitry Werner f33c5e9d72
KAFKA-19512 Add missing MVs in MetadataVersionTest (#20173)
Add missing MVs in MetadataVersionTest.

Reviewers: Jun Rao <junrao@gmail.com>
2025-07-17 14:41:18 -07:00
Gaurav Narula 12761c07ae
KAFKA-19458: resume cleaning on future replica dir change (#20082)
`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>
2025-07-17 13:13:09 -07:00
Calvin Liu 9412051dc6
MINOR: Bump LATEST_PRODUCTION to 4.1IV1 and Use MV to enable ELR (#20137)
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>
2025-07-17 11:53:10 -07:00
Logan Zhu d03878c7fb
MINOR: Migrate CoordinatorLoaderImpl from Scala to Java (#20089)
CI / build (push) Waiting to run Details
### 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>
2025-07-18 01:51:46 +08:00
Chang-Chi Hsu 4a2d4ee76a
MINOR: Replace Long with primitive long for CoordinatorPlayback (#20171)
Reviewers: Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping
 Tsai <chia7712@gmail.com>
2025-07-18 01:34:00 +08:00
Chang-Chi Hsu 5ac0266879
MINOR: Declare inner RocksDBDualCFIterator class as static (#20182)
Reviewers: Lan Ding <isDing_L@163.com>, Ken Huang <s7133700@gmail.com>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-07-17 20:54:34 +08:00
KTKTK-HZ e8e4e0b6db
KAFKA-19377:Update /streams/developer-guide/security.html for KIP-1071 (#20084)
CI / build (push) Waiting to run Details
Added required ACLs for new streams operations:

- STREAMS_GROUP_HEARTBEAT (88) requires:
  • READ on Group
  • DESCRIBE on Topics
  • [Conditional] CREATE on Cluster or Topics
- STREAMS_GROUP_DESCRIBE (89) requires:
  • DESCRIBE on Group
  • DESCRIBE on Topic

Here is the rendering of the modified document.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
Co-authored-by: Lucas Brutschy <lbrutschy@gmail.com>
2025-07-17 10:35:46 +02:00
Federico Valeri f2cbc7e3f2
MINOR: KafkaRaftClient cleanup (#20161)
Various minor cleanups for the KRaft client.

---------

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>
2025-07-17 10:41:28 +08:00
Gaurav Narula 7e9df7d03d
KAFKA-19505: allow mocking UnifiedLog#topicId in ReplicaManagerTest (#20167)
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>
2025-07-17 10:40:00 +08:00
Tsung-Han Ho (Miles Ho) 70824be92a
KAFKA-19501 Update OpenJDK base image from buster to bullseye (#20165)
CI / build (push) Waiting to run Details
The changes update the OpenJDK base image from 17-buster to 17-bullseye:
- Updates tests/docker/Dockerfile to use openjdk:17-bullseye instead of
openjdk:17-buster
  - Updates tests/docker/ducker-ak script to use the new default image
- Updates documentation in tests/README.md with the new image name
examples

Reviewers: Federico Valeri <fedevaleri@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-07-17 03:27:07 +08:00
Ming-Yen Chung e3276ae029
KAFKA-19427 Allow the coordinator to grow its buffer dynamically (#20040)
CI / build (push) Waiting to run Details
* Coordinator starts with a smaller buffer, which can grow as needed.

* In freeCurrentBatch, release the appropriate buffer:
  * The Coordinator recycles the expanded buffer
(`currentBatch.builder.buffer()`), not `currentBatch.buffer`, because
`MemoryBuilder` may allocate a new `ByteBuffer` if the existing one
isn't large enough.

  * There are two cases that buffer may exceeds `maxMessageSize`      1.
If there's a single record whose size exceeds `maxMessageSize` (which,
so far, is derived from `max.message.bytes`) and the write is in
`non-atomic` mode, it's still possible for the buffer to grow beyond
`maxMessageSize`. In this case, the Coordinator should revert to using a
smaller buffer afterward.      2. Coordinator do not recycles the buffer
that larger than `maxMessageSize`. If the user dynamically reduces
`maxMessageSize` to a value even smaller than `INITIAL_BUFFER_SIZE`, the
Coordinator should avoid recycling any buffer larger than
`maxMessageSize` so that Coordinator can allocate the smaller buffer in
the next round.

* Add tests to verify the above scenarios.

Reviewers: David Jacot <djacot@confluent.io>, Sean Quah
<squah@confluent.io>, Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, TaiJuWu <tjwu1217@gmail.com>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-07-16 22:06:33 +08:00
Sanskar Jhajharia 65a9337739
MINOR: Add ShareFetch quota session verification test (#20164)
CI / build (push) Waiting to run Details
### 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>
2025-07-16 09:56:01 +01:00
Chang-Chi Hsu 2658f25238
KAFKA-19305 Make ClientQuotaImage and TopicImage immutable (#19847)
- Updated `ClientQuotaImage` and `TopicImage` by using
`Collections.unmodifiableMap` or `ImmutableMap` to prevent accidental or
intentional mutations after construction.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-07-16 11:50:18 +08:00
Masahiro Mori daece61a50
MINOR: Refactor LockUtils and improve comments (follow up to KAFKA-19390) (#20131)
CI / build (push) Waiting to run Details
This PR performs a refactoring of LockUtils and improves inline
comments, as a follow-up to https://github.com/apache/kafka/pull/19961.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
2025-07-15 10:07:01 -07:00
lucliu1108 7ea32a0e93
KAFKA-19459: List internal topics for the user (#20157)
CI / build (push) Waiting to run Details
For the Kafka Stream group commands, if delete topic requests fail due
to version mismatch, user will have to remove the topics manually by
first retrieving the relevant internal topics.

To assist the user, the internal topic names are now included as part of
the error message, so that the user could delete the internal topics
associated with this application directly.

Reviewers: TengYao Chi <frankvicky@apache.org>, Alieh Saeedi
<asaeedi@confluent.io>
2025-07-15 11:52:35 +08:00
Bill Bejeck dd82542493
KAFKA-19504: Remove unused metrics reporter initialization in KafkaAdminClient (#20166)
CI / build (push) Waiting to run Details
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>
2025-07-14 20:19:16 -04:00
Matthias J. Sax ffcfc974d9
KAFKA-19842: Fix flaky KafkaStreamsTelemetryIntegrationTest (#20147)
The new "streams" protocol behaves slightly different to the "classic"
protocol, and thus we need to update the test to avoid race conditions.
In particular, the first call to `poll()` won't "block" and return after
task assignment completed if we need to create internal topics,  but
returns early without a task assignment, and only a consecutive
rebalance will assign tasks.

This implies, that KafkaStreams transits to RUNNING state even if the
group is still in NOT_READY state broker side, but this NOT_READY  state
is not reflected in the client side state machine.

Disabling the combination of "complex-topology + streams" for now,
until this difference in behavior of the client state machine is fixed.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-07-14 17:13:54 -07:00
Xuan-Zhang Gong c6cf5175f6
MINOR: Add a description document for batchLength (#20140)
Add documentation for Batch Format to explain the meaning of
batchLength.

This is the preview image after the change:


![image](https://github.com/user-attachments/assets/85023c48-64e6-4a33-898f-df84f6864e58)

Reviewers: Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-07-15 05:51:24 +08:00