Commit Graph

314 Commits

Author SHA1 Message Date
Kevin Wu 012e4ca6d8
KAFKA-19719 --no-initial-controllers should not assume kraft.version=1 (#20604)
CI / build (push) Has been cancelled Details
```
commit ec37eb538b (HEAD ->
KAFKA-19719-cherry-pick-41, origin/KAFKA-19719-cherry-pick-41)
Author: Kevin Wu <kevin.wu2412@gmail.com>
Date:   Thu Sep 25 11:56:16 2025 -0500

    KAFKA-19719: --no-initial-controllers should not assume
kraft.version=1 (#20551)

    Just because a controller node sets --no-initial-controllers flag
does     not mean it is necessarily running kraft.version=1. The more
precise     meaning is that the controller node being formatted does not
know what     kraft version the cluster should be in, and therefore it
is only safe to     assume kraft.version=0. Only by setting
--standalone,--initial-controllers, or --no-initial-controllers     AND
not specifying the controller.quorum.voters static config, is it
known kraft.version > 0.

    For example, it is a valid configuration (although confusing) to run
a     static   quorum defined by controller.quorum.voters but have all
the     controllers   format with --no-initial-controllers. In this
case,     specifying --no-initial-controllers alongside a metadata
version that     does not  support kraft.version=1 causes formatting to
fail, which is     does not  support kraft.version=1 causes formatting
to fail, which is     a  regression.

    Additionally, the formatter should not check the kraft.version
against     the release version, since kraft.version does not actually
depend on any     release version. It should only check the
kraft.version against the     static voters config/format arguments.

    This PR also cleans up the integration test framework to match the
semantics of formatting an actual cluster.

    Reviewers: TengYao Chi <kitingiao@gmail.com>, Kuan-Po Tseng
<brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, José Armando
García Sancio <jsancio@apache.org>      Conflicts:
core/src/main/scala/kafka/tools/StorageTool.scala Minor conflicts. Keep
changes from cherry-pick.
core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java
Remove auto-join tests, since 4.1 does not support it. docs/ops.html
Keep docs section from cherry-pick.
metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java
Minor conflicts. Keep cherry-picked changes.
test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java
Conflicts due to integration test framework changes. Keep new changes.

commit 02d58b176c (upstream/4.1)
```

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-30 22:30:23 +08:00
Masahiro Mori 7f8c2a5fc9 MINOR: Refactor LockUtils and improve comments (follow up to KAFKA-19390) (#20131)
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-09-30 21:27:14 +08:00
Masahiro Mori 807866ca6d KAFKA-19390: Call safeForceUnmap() in AbstractIndex.resize() on Linux to prevent stale mmap of index files (#19961)
https://issues.apache.org/jira/browse/KAFKA-19390

The AbstractIndex.resize() method does not release the old memory map
for both index and time index files.  In some cases, Mixed GC may not
run for a long time, which can cause the broker to crash when the
vm.max_map_count limit is reached.

The root cause is that safeForceUnmap() is not being called on Linux
within resize(), so we have changed the code to unmap old mmap on all
operating systems.

The same problem was reported in
[KAFKA-7442](https://issues.apache.org/jira/browse/KAFKA-7442), but the
PR submitted at that time did not acquire all necessary locks around the
mmap accesses and was closed without fixing the issue.

Reviewers: Jun Rao <junrao@gmail.com>
2025-09-30 21:26:59 +08:00
Calvin Liu 98cb8df7a5
MINOR: Bump LATEST_PRODUCTION to 4.1IV1 and Use MV to enable ELR (#20174)
CI / build (push) Waiting to run Details
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: Jun Rao <junrao@gmail.com>
2025-07-15 20:23:53 -07:00
José Armando García Sancio 88eced0c0f KAFKA-14145; Faster KRaft HWM replication (#19800)
This change compares the remote replica's HWM with the leader's HWM and
completes the FETCH request if the remote HWM is less than the leader's
HWM. When the leader's HWM is updated any pending FETCH RPC is
completed.

Reviewers: Alyssa Huang <ahuang@confluent.io>, David Arthur
 <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>
(cherry picked from commit 742b327025)
2025-06-17 13:27:11 -04:00
Apoorv Mittal d07aa37412
KAFKA-19386: Correcting ExpirationReaper thread names from Purgatory (#19918)
The PR: https://github.com/apache/kafka/pull/17636 migrated
DelayedOperationPurgatory from scala to java, and instatiated
`expirationReaper` at instance level where `purgatoryName` is still
`null` hence all expiration threads from different Purgatories has
incorrect names.

<img width="216" alt="Screenshot 2025-06-07 at 01 28 58"

src="https://github.com/user-attachments/assets/fd1b8137-b290-42e0-9a95-258fde5737d2"
/>

The PR fixes the instatiation of ExpirationReaper, in constructor when
`purgatoryName` is defined.

<img width="296" alt="Screenshot 2025-06-07 at 01 31 27"

src="https://github.com/user-attachments/assets/9912311b-ddf6-4554-8e04-d0b8ad208abc"
/>

This issue affects 4.0 version as well, though minor.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-09 12:10:59 +01:00
Okada Haruki e2500186cb
KAFKA-19334 MetadataShell execution unintentionally deletes lock file (#19817)
## 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>
2025-06-09 12:24:26 +08:00
Andrew Schofield 223684bad1
KAFKA-16894: share.version becomes stable feature for preview (#19831)
This PR sets `SV_1` as the latest-production share version. This means
for AK 4.1 it will be a preview feature, not enabled by default, but can
be enabled without turning on unstable features. This is analogous to
how ELR worked for AK 4.0.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-02 13:45:37 +01:00
S.Y. Wang 543fb6c848
KAFKA-19336 Upgrade Jackson to 2.19.0 (#19835)
`JsonNode.fields()` method has been deprecated by 
- https://github.com/FasterXML/jackson-databind/issues/4863
- https://github.com/FasterXML/jackson-databind/pull/4871

So modified accordingly.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-28 20:53:43 +08:00
YuChia Ma 6e380fbbcc
KAFKA-19322 Remove the DelayedOperation constructor that accepts an external lock (#19798)
CI / build (push) Waiting to run Details
Remove the DelayedOperation constructor that accepts an external lock.

According to this [PR](https://github.com/apache/kafka/pull/19759).

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-27 01:05:41 +08:00
Ken Huang bcda92b5b9
KAFKA-19080 The constraint on segment.ms is not enforced at topic level (#19371)
CI / build (push) Waiting to run Details
The main issue was that we forgot to set
`TopicConfig.SEGMENT_BYTES_CONFIG` to at least `1024 * 1024`, which
caused problems in tests with small segment sizes.

To address this, we introduced a new internal config:
`LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG`, allowing us to set smaller
segment bytes specifically for testing purposes.

We also updated the logic so that if a user configures the topic-level
segment bytes without explicitly setting the internal config, the
internal value will no longer be returned to the user.

In addition, we removed
`MetadataLogConfig#METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG` and added
three new internal configurations:
- `INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_DELETE_DELAY_MILLIS_CONFIG`

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-25 20:57:22 +08:00
Apoorv Mittal adb76779ed
KAFKA-19312 Avoiding concurrent execution of onComplete and tryComplete (#19759)
The `onComplete` method in DelayedOperation is guaranteed to run only
once, through `forceComplete`, invoked either by `tryComplete` or when
operation is expired (`run` method). The invocation of  `tryComplete` is
done by attaining `lock` so no concurrent execution of  `tryComplete`
happens for same delayed operation. However, there can be  concurrent
execution of `tryComplete` and `onComplete` as the  `expiration` thread
can trigger a separte run of `onComplete` while  `tryComplete` is still
executing. This behaviour is not ideal as there  are parallel runs where
1 threads method execution is wasteful i.e. if  `onComplete` is already
invoked by another thread then execution of  `tryComplete` is not
required.

I ran some tests and performance is same.

### After the chages:

```
--num 10000 --rate 100 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 7
# interval samples: rate = 100.068948, min = 0, max = 129
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
101196	99.809364	99.806376	3240	0 2	0 8
```

```
--num 10000 --rate 1000 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 9
# interval samples: rate = 999.371395, min = 0, max = 14
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
11104	989.902990	989.805008	1349	0 2	0 7
```

### Before changes:

```
--num 10000 --rate 100 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 9
# interval samples: rate = 100.020304, min = 0, max = 130
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
102366	98.657274	98.652408	3444	0 2	0 8

--num 10000 --rate 1000 --timeout 1000 --pct50 0.5 --pct75 0.75

# latency samples: pct75 = 0, pct50 = 0, min = 0, max = 8
# interval samples: rate = 997.134236, min = 0, max = 14
# enqueue rate (10000 requests):
# <elapsed time ms>	<target rate>	<actual rate>	<process cpu time ms>	<G1 Old Generation count> <G1 Young Generation count>	<G1 Old Generation time ms> <G1 Young Generation time ms>
11218	978.665101	978.665101	1624	0 2	0 7

Reviewers: Jun Rao <junrao@gmail.com>, Andrew Schofield
 <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-25 14:36:43 +08:00
Ken Huang bbe27e65a3
MINOR: Fix the CLIENT_QUOTA_CALLBACK_CLASS_CONFIG document (#18713)
See the discussion:

7fa0dfd72d (r1929621640)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-23 23:49:26 +08:00
jimmy b44bfca408
KAFKA-16717 [2/N]: Add AdminClient.alterShareGroupOffsets (#18929)
[KAFKA-16720](https://issues.apache.org/jira/browse/KAFKA-16720) aims to
finish the AlterShareGroupOffsets RPC.

Reviewers: Andrew Schofield <aschofield@confluent.io>

---------

Co-authored-by: jimmy <wangzhiwang@qq.com>
2025-05-23 09:05:48 +01:00
Chirag Wadhwa f9064f8bcd
KAFKA-19231-1: Handle fetch request when share session cache is full (#19701)
According to the recent changes in KIP-932, when the share session cache
is full and a broker receives a Share Fetch request with Initial Share
Session Epoch (0), then the error code `SHARE_SESSION_LIMIT_REACHED` is
returned after a delay of maxWaitMs. This PR implements this logic. In
order to add a delay between subsequent share fetch requests, the timer
is delayed operation purgatory is used. A new `IdleShareFetchTimeTask`
has been added which takes in a CompletableFuture<Void>. Upon the
expiration, this future is completed with null. When the future is
completes, a response is sent back to the client with the error code
`SHARE_SESSION_LIMIT_REACHED`

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-15 14:36:44 +01:00
YuChia Ma 1c3c361629
KAFKA-19263 update `delete.topic.enable` docs (#19675)
If delete.topic.enable is false on the brokers, deleteTopics will mark
the topics for deletion, but not actually delete them. The futures will
return successfully in this case.

It is not true as the server return exception now.

```java if (!config.deleteTopicEnable) {   if (apiVersion < 3) { throw
new InvalidRequestException("Topic deletion is disabled.")    } else {
throw new TopicDeletionDisabledException()    } } ```

Reviewers: Nick Guo <lansg0504@gmail.com>, PoAn Yang
 <payang@apache.org>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-15 20:03:54 +08:00
Dmitry Werner 4f2e3ecad4
MINOR: Remove unused TopicOptionalIdPartition class (#19716)
After merging the

[commit](6f783f8536 (diff-78812e247ffeae6f8c49b1b22506434701b1e1bafe7f92ef8f8708059e292bf0L53)),
the `TopicOptionalIdPartition` class is no longer used anywhere and
should be removed.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-14 20:52:49 +08:00
Andrew Schofield 7d027a4d83
KAFKA-19218: Add missing leader epoch to share group state summary response (#19602)
CI / build (push) Waiting to run Details
When the persister is responding to a read share-group state summary
request, it has no way of including the leader epoch in its response,
even though it has the information to hand. This means that the leader
epoch information is not initialised in the admin client operation to
list share group offsets, and this then means that the information
cannot be displayed in kafka-share-groups.sh.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan
 <smahajan@confluent.io>
2025-05-06 14:53:12 +01:00
Abhinav Dixit caf4a6cc5f
KAFKA-19216: Eliminate flakiness in kafka.server.share.SharePartitionTest (#19639)
### About
11 of the test cases in `SharePartitionTest` have failed at least once
in the past 28 days.

https://develocity.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.timeZoneId=Europe%2FLondon&tests.container=kafka.server.share.SharePartitionTest
Observing the flakiness, they seem to be caused due to the usage of
`SystemTimer` for various acquisition lock timeout related tests. I have
replaced the usage of `SystemTimer` with `MockTimer` and also improved
the `MockTimer` API with regard to removing the timer task entries that
have already been cancelled.
Also, this has reduced the time taken to run `SharePartitionTest` from
~6 sec to ~1.5 sec

### Testing
The testing has been done with the help of already present unit tests in
Apache Kafka.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-05 20:04:22 +01:00
Jhen-Yung Hsu 014d0186cc
MINOR: Move AdminCommandFailedException and AdminOperationException to tools module (#19614)
CI / build (push) Waiting to run Details
AdminCommandFailedException and AdminOperationException are used only in
tools module, so move both into tools module.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-02 11:25:12 +08:00
YuChia Ma 979f49f967
KAFKA-19146 Merge OffsetAndEpoch from raft to server-common (#19475)
CI / build (push) Waiting to run Details
1. remove org.apache.kafka.raft.OffsetAndEpoch
2. rewrite org.apache.kafka.server.common.OffsetAndEpoch by record
keyword
3. rename OffsetAndEpoch#leaderEpoch to OffsetAndEpoch#epoch

Reviewers: PoAn Yang <payang@apache.org>, Xuan-Zhang Gong
 <gongxuanzhangmelt@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-02 03:12:52 +08:00
Andrew Schofield 2022b4c480
KAFKA-16894 Correct definition of ShareVersion (#19606)
The ShareVersion feature does not make any metadata version changes. As
a result, `SV_1` does not depend on any MV level, and no MV needs to be
defined for the preview of KIP-932.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-02 01:30:00 +08:00
Matthias J. Sax b0a26bc2f4
KAFKA-19173: Add `Feature` for "streams" group (#19509)
Add new StreamsGroupFeature, disabled by default,  and add "streams" as
default value to `group.coordinator.rebalance.protocols`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot
<david.jacot@gmail.com>, Lucas Brutschy <lbrutschy@confluent.io>,
Justine Olshan <jolshan@confluent.io>, Andrew Schofield
<aschofield@confluent.io>, Jun Rao <jun@confluent.io>
2025-04-29 22:51:10 -07:00
PoAn Yang 81881dee83
KAFKA-18760: Deprecate Optional<String> and return String from public Endpoint#listener (#19191)
* Deprecate org.apache.kafka.common.Endpoint#listenerName.
* Add org.apache.kafka.common.Endpoint#listener to replace
org.apache.kafka.common.Endpoint#listenerName.
* Replace org.apache.kafka.network.EndPoint with
org.apache.kafka.common.Endpoint.
* Deprecate org.apache.kafka.clients.admin.RaftVoterEndpoint#name
* Add org.apache.kafka.clients.admin.RaftVoterEndpoint#listener to
replace org.apache.kafka.clients.admin.RaftVoterEndpoint#name

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao
 Chi <frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>, Bagda
 Parth  , Kuan-Po Tseng <brandboat@gmail.com>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-04-30 12:15:33 +08:00
José Armando García Sancio b97a130c08
KAFKA-16538; Enable upgrading kraft version for existing clusters (#19416)
This change implements upgrading the kraft version from 0 to 1 in existing clusters.
Previously, clusters were formatted with either version 0 or version 1, and could not
be moved between them.

The kraft version for the cluster metadata partition is recorded using the
KRaftVersion control record. If there is no KRaftVersion control record
the default kraft version is 0.

The kraft version is upgraded using the UpdateFeatures RPC. These RPCs
are handled by the QuorumController and FeatureControlManager. This
change adds special handling in the FeatureControlManager so that
upgrades to the kraft.version are directed to
RaftClient#upgradeKRaftVersion.

To allow the FeatureControlManager to call
RaftClient#upgradeKRaftVersion is a non-blocking fashion, the kraft
version upgrade uses optimistic locking. The call to
RaftClient#upgradeKRaftVersion does validations of the version change.
If the validations succeeds, it generates the necessary control records
and adds them to the BatchAccumulator.

Before the kraft version can be upgraded to version 1, all of the
brokers and controllers in the cluster need to support kraft version 1.
The check that all brokers support kraft version 1 is done by the
FeatureControlManager. The check that all of the controllers support
kraft version is done by KafkaRaftClient and LeaderState.

When the kraft version is 0, the kraft leader starts by assuming that
all voters do not support kraft version 1. The leader discovers which
voters support kraft version 1 through the UpdateRaftVoter RPC. The
KRaft leader handles UpdateRaftVoter RPCs by storing the updated
information in-memory until the kraft version is upgraded to version 1.
This state is stored in LeaderState and contains the latest directory
id, endpoints and supported kraft version for each voter.

Only when the KRaft leader has received an UpdateRaftVoter RPC from all
of the voters will it allow the upgrade from kraft.version 0 to 1.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-04-22 16:02:51 -07:00
Andrew Schofield 66147d5de7
KAFKA-19057: Stabilize KIP-932 RPCs for AK 4.1 (#19378)
This PR removes the unstable API flag for the KIP-932 RPCs.

The 4 RPCs which were exposed for the early access release in AK 4.0 are
stabilised at v1. This is because the RPCs have evolved over time and AK
4.0 clients are not compatible with AK 4.1 brokers. By stabilising at
v1, the API version checks prevent incompatible communication and
server-side exceptions when trying to parse the requests from the older
clients.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-04-22 11:43:32 +01:00
Kamal Chandraprakash 2cd733c9b3
KAFKA-17184: Fix the error thrown while accessing the RemoteIndexCache (#19462)
For segments that are uploaded to remote, RemoteIndexCache caches the
fetched offset, timestamp, and transaction index entries on the first
invocation to remote, then the subsequent invocations are accessed from
local.

The remote indexes that are cached locally gets removed on two cases:

1. Remote segments that are deleted due to breach by retention size/time
and start-offset.
2. The number of cached indexes exceed the remote-log-index-cache size
limit of 1 GB (default).

There are two layers of locks used in the RemoteIndexCache. First-layer
lock on the RemoteIndexCache and the second-layer lock on the
RemoteIndexCache#Entry.

**Issue**

1. The first-layer of lock coordinates the remote-log reader and deleter
threads. To ensure that the reader and deleter threads are not blocked
on each other, we only take `lock.readLock()` when accessing/deleting
the cached index entries.
2. The issue happens when both the reader and deleter threads took the
readLock, then the deleter thread marked the index as
`markedForCleanup`. Now, the reader thread which holds the `indexEntry`
gets an IllegalStateException when accessing it.
3. This is a concurrency issue, where we mark the entry as
`markedForCleanup` before removing it from the cache. See
RemoteIndexCache#remove, and RemoteIndexCache#removeAll methods.
4. When an entry gets evicted from cache due to breach by maxSize of 1
GB, then the cache remove that entry before calling the evictionListener
and all the operations are performed atomically by caffeine cache.

**Solution**

1. When the deleter thread marks an Entry for deletion, then we rename
the underlying index files with ".deleted" as suffix and add a job to
the remote-log-index-cleaner thread which perform the actual cleanup.
Previously, the indexes were not accessible once it was marked for
deletion. Now, we allow to access those renamed files (from entry that
is about to be removed and held by reader thread) until those relevant
files are removed from disk.
2. Similar to local-log index/segment deletion, once the files gets
renamed with ".deleted" as suffix then the actual deletion of file
happens after `file.delete.delay.ms` delay of 1 minute. The renamed
index files gets deleted after 30 seconds.
3. During this time, if the same index entry gets fetched again from
remote, then it does not have conflict with the deleted entry as the
file names are different.

Reviewers: Satish Duggana <satishd@apache.org>
2025-04-18 16:43:37 +05:30
Mickael Maison fb2ce76b49
KAFKA-18888: Add KIP-877 support to Authorizer (#19050)
This also adds metrics to StandardAuthorizer

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>
2025-04-15 19:40:24 +02:00
Xuan-Zhang Gong fc25436440
MINOR: Modify KafkaEventQueue VoidEvent to as singleton and use more proper function interface (#19356)
class `VoidEvent` provides singleton object , but nobody use it.  I
think we should private `VoidEvent` constructor and only use singleton.
use `UnaryOperator<OptionalLong>` instead
`Function<OptionalLong,OptionalLong>`

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-14 17:35:24 +08:00
Dmitry Werner 7863b35064
KAFKA-14485: Move LogCleaner to storage module (#19387)
Move LogCleaner and related classes to storage module and rewrite in
Java.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Jun Rao <junrao@gmail.com>
2025-04-11 09:21:05 -07:00
Andrew Schofield 21a080f08c
KAFKA-16894: Define feature to enable share groups (#19293)
This PR proposes a switch to enable share groups for 4.1 (preview) and
4.2 (GA).

* `share.version=1` to indicate that share groups are enabled. This is
used as the switch for turning share groups on and off.

In 4.1, the default will be `share.version=0`. Then a user wanting to
evaluate the preview of KIP-932 would use `bin/kafka-features.sh
--bootstrap.server xxxx upgrade --feature share.version=1`.

In 4.2, the default will be `share.version=1`.

Reviewers: Jun Rao <junrao@gmail.com>
2025-04-11 12:14:38 +01:00
Chirag Wadhwa 5148174196
KAFKA-16718-2/n: KafkaAdminClient and GroupCoordinator implementation for DeleteShareGroupOffsets RPC (#18976)
This PR contains the implementation of KafkaAdminClient and
GroupCoordinator for DeleteShareGroupOffsets RPC.

- Added `deleteShareGroupOffsets` to `KafkaAdminClient`
- Added implementation for `handleDeleteShareGroupOffsetsRequest` in
`KafkaApis.scala`
- Added `deleteShareGroupOffsets` to `GroupCoordinator` as well.
internally this makes use of `persister.deleteState` to persist the
changes in share coordinator

Reviewers: Andrew Schofield <aschofield@confluent.io>, Sushant Mahajan <smahajan@confluent.io>
2025-04-09 07:31:06 +01:00
Nick Guo a4ea5dff0d
KAFKA-19099 Remove GroupSyncKey, GroupJoinKey, and MemberKey (#19413)
They are useless after removing old coordinator.

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-04-09 01:51:32 +08:00
Xuan-Zhang Gong 9a2b8b6025
MINOR: Move DelayedRecords purgatory classes to server (#19408)
These classes are only used by server classes so they don't need to be
in the server-common module.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, PoAn Yang
<poan.yang@suse.com>
2025-04-08 11:30:31 +02:00
TengYao Chi 6d68f8a82c
MINOR: Move BrokerReconfigurable to the sever-common module (#19383)
This patch moves `BrokerReconfigurable` to the `server-common module`
and decouples the `TransactionLogConfig` and `KafkaConfig` to unblock
KAFKA-14485.

Reviewers: PoAn Yang <payang@apache.org>, TaiJuWu <tjwu1217@gmail.com>,
Chia-Ping Tsai <chia7712@gmail.com>
2025-04-07 07:39:01 +08:00
Ming-Yen Chung 8c77953d5f
MINOR: Revert "migrate LogFetchInfo, Assignment and RequestAndCompletionHandler to java record" (#19177)
Revert some java record migration in #19062  #18783 

We assume java record is purely immutable data carriers.
As discussed in
https://github.com/apache/kafka/pull/19062#issuecomment-2709637352, if a
class has fields that may be mutable, we shouldn't migrate it to Java
record because the hashcode/equals behavior are changed.

* LogFetchInfo (Records)
* Assignment (successCallback)
* Remove `equals` method from Assignment since `Assignment` is not and
shouldn't be used in Map/Set key.
* RequestAndCompletionHandler (handler)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-05 23:01:17 +08:00
Gaurav Narula 25f9b1ee4c
MINOR: ShutdownableThread: log on error level on FatalExitError (#19351)
This will help debugging the error as the stacktrace is valuable in identifying the origin.

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Igor Soarez <i@soarez.me>, Ken Huang <s7133700@gmail.com>
2025-04-04 15:38:45 +01:00
Xuan-Zhang Gong 2994e5eff3
KAFKA-19004 Move DelayedDeleteRecords to server-common module (#19226)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-03 00:06:27 +08:00
Sean Quah 7ee73e6cd5
MINOR: Accept Throwables in DeferredEventQueue.failAll (#19337)
DeferredEvents take a Throwable and not an Exception. For consistency,
DeferredEventQueue.failAll should also accept a Throwable.

Reviewers: David Jacot <djacot@confluent.io>
2025-04-01 05:25:13 -07:00
Dmitry Werner 84b8fec089
KAFKA-14486 Move LogCleanerManager to storage module (#19216)
Move LogCleanerManager and related classes to storage module and rewrite
in Java.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Jun Rao
<junrao@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
2025-03-27 12:35:38 +08:00
Sushant Mahajan eb88e78373
KAFKA-18827: Initialize share group state group coordinator impl. [3/N] (#19026)
* This PR adds impl for the initialize share groups call from the Group
Coordinator perspective.
* The initialize call on persister instance will be invoked by the
`GroupCoordinatorService`, based on the response of the
`GroupCoordinatorShard.shareGroupHeartbeat`. If there is new topic
subscription or member assignment change (topic paritions incremented),
the delta share partitions corresponding to the share group in question
are returned as an optional initialize request.
* The request is then sent to the share coordinator as an encapsulated
timer task because we want the heartbeat response to go asynchronously.
* Tests have been added for `GroupCoordinatorService` and
`GroupMetadataManager`. Existing tests have also been updated.
* A new formatter `ShareGroupStatePartitionMetadataFormatter` has been
added for debugging.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-26 19:40:23 +00:00
Sanskar Jhajharia 15474e0100
MINOR: Cleanup Server Common Module (#19085)
Given that now we support Java 17 on our brokers, this PR replace the
use of the following in server-common module:

- Collections.singletonList() and Collections.emptyList() with List.of()
- Collections.singletonMap() and Collections.emptyMap() with Map.of()
- Collections.singleton() and Collections.emptySet() with Set.of()
- Arrays.asList() with List.of()

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-25 01:14:00 +08:00
TengYao Chi 20bad6efb3
KAFKA-18576 Convert ConfigType to Enum (#18711)
JIRA: KAFKA-18576
After removing ZooKeeper, we no longer need to exclude `client_metrics`
and `group` from `ConfigType#ALL`.

Since it's a common pattern to provide a mechanism to know all values in
enumeration ( Java enum provides ootb), we should convert ConfigType to
enum.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-25 01:10:59 +08:00
PoAn Yang d497250c22
KAFKA-18999 Remove BrokerMetadata (#19227)
* Replace `BrokerMetadata` with `UsableBroker` in KRaftMetadataCache and
ReassignPartitionsCommand.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-22 19:30:28 +08:00
José Armando García Sancio d04efca493
KAFKA-18979; Report correct kraft.version in ApiVersions (#19205)
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>
2025-03-13 18:39:24 -04:00
Apoorv Mittal f3da8f500e
KAFKA-18936: Fix share fetch when records are larger than max bytes (#19145)
The PR fixes the behaviour when records are fetched which are larger
than `fetch.max.bytes` config.

The usage of `hardMaxBytesLimit` is in ReplicaManager where it decides
whether to fetch a single record or not. The file records get sliced
based on the bytes requested. However, if `hardMaxBytesLimit` is false
then at least one record is fetched and bytes are adjusted accordingly in
`localLog`.

Reviewers: Jun Rao <junrao@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-03-12 09:03:35 +00:00
Lucas Brutschy fc2e3dfce9
MINOR: Disallow unused local variables (#18963)
Recently, we found a regression that could have been detected by static
analysis, since a local variable wasn't being passed to a method during
a refactoring, and was left unused. It was fixed in
[7a749b5](7a749b589f),
but almost slipped into 4.0. Unused variables are typically detected by
IDEs, but this is insufficient to prevent these kinds of bugs. This
change enables unused local variable detection in checkstyle for Kafka.

A few notes on the usage:
- There are two situations in which people actually want to have a local
variable but not use it. First, there are `for (Type ignored:
collection)` loops which have to loop `collection.length` number of
times, but that do not use `ignored` in the loop body. These are
typically still easier to read than a classical `for` loop. Second, some
IDEs detect it if a return value of a function such as `File.delete` is
not being used. In this case, people sometimes store the result in an
unused local variable to make ignoring the return value explicit and to
avoid the squiggly lines.
- In Java 22, unsued local variables can be omitted by using a single
underscore `_`. This is supported by checkstyle. In pre-22 versions,
IntelliJ allows such variables to be named `ignored` to suppress the
unused local variable warning. This pattern is often (but not
consistently) used in the Kafka codebase. This is, however, not
supported by checkstyle.

Since we cannot switch to Java 22, yet, and we want to use automated
detection using checkstyle, we have to resort to prefixing the unused
local variables with `@SuppressWarnings("UnusedLocalVariable")`. We have
to apply this in 11 cases across the Kafka codebase. While not being
pretty, I'd argue it's worth it to prevent bugs like the one fixed in
[7a749b5](7a749b589f).

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur
<mumrah@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bruno
Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>
2025-03-10 09:37:35 +01:00
Colin Patrick McCabe 343bc995f4
KAFKA-18920: The kcontrollers must set kraft.version in ApiVersionsResponse (#19127)
The kafka controllers need to set kraft.version in their
ApiVersionsResponse messages according to the current kraft.version
reported by the Raft layer. Instead, currently they always set it to 0.

Also remove FeatureControlManager.latestFinalizedFeatures. It is not
needed and it does a lot of copying.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-07 13:46:46 -08:00
leaf-soba 53b2935c51
KAFKA-18461 Add Objects.requireNotNull to Snapshot (#18684)
Add explicit not-null checks in Snapshot so we get a better error message in the event that a Snapshot object is accessed after erase has been called.

Reviewers: David Arthur <mumrah@gmail.com>
2025-03-07 09:48:09 -05:00
co63oc 3d7ac0c3d1
MINOR: Fix typos in multiple files (#19102)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-05 14:27:32 +00:00