Commit Graph

16080 Commits

Author SHA1 Message Date
José Armando García Sancio 742b327025
KAFKA-14145; Faster KRaft HWM replication (#19800)
CI / build (push) Waiting to run Details
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>
2025-06-17 13:00:43 -04:00
S.Y. Wang bf15205647
KAFKA-19350 Don't propagate the error caused by CreateTopicPolicy to FatalFaultHandler (#19857)
`CreateTopicPolicy#validate` may throw unexpected exception  other than
`PolicyViolationException`. We should handle this case as well.

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-18 00:51:50 +08:00
Ken Huang 91ce182ec5
KAFKA-19042 Move ProducerSendWhileDeletionTest to client-integration-tests module (#19971)
Use Java to rewrite ProducerSendWhileDeletionTest by new test infra and
move it to client-integration-tests module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-18 00:47:30 +08:00
Lan Ding 183ad524dc
MINOR: replace test "catch exception" by assertThrows (#19975)
see https://github.com/apache/kafka/pull/19948#discussion_r2150617216,
replace test "catch exception" by assertThrows.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-18 00:07:49 +08:00
Kuan-Po Tseng 86cd5d50f5
KAFKA-14895: [2/N] Move AddPartitionsToTxnManager files to java (#19933)
Move AddPartitionsToTxnManagerTest to java

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-17 22:56:19 +08:00
Lucas Brutschy 31a7e01769
KAFKA-19412: Extended AuthorizerIntegrationTest to cover StreamsGroupHeartbeat (#19978)
Extending test coverage of authorization for streams group RPC
StreamsGroupHeartbeat. The RPC requires READ GROUP and DESCRIBE TOPIC
permissions for all topics. For creating internal topics, we require
CREATE TOPIC permission. If internal topic  creation fails, the request
does not fail, but the status reflects this problem.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2025-06-17 16:41:49 +02:00
PoAn Yang a94f7caf76
MINOR: add lower case lister name integration test (#19932)
In [KIP-1143](https://cwiki.apache.org/confluence/x/LwqWF), it
deprecated Endpoint#listenerName and removed
org.apache.kafka.network.EndPoint. Certain parts of the code depend on
listener name normalization. We should add a test to make sure there is
no regression.

Followup: 
https: //github.com/apache/kafka/pull/19191#issuecomment-2939855317
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-17 22:41:31 +08:00
Dave Troiano 6f946d5026
MINOR: in min.insync.replicas config doc, explicitly state that all ISR must ack when acks=all (#19973)
CI / build (push) Waiting to run Details
Clarify the interaction of `min.insync.replicas` and `ack=all`
configuration.  Prior to this change, the doc for `min.insync.replicas`
could have been  interpreted as being used to short-circuit in the
`acks=all` case as if  it would be enough if `min.inscyn.replicas`
number of brokers replicated  a message before it can be acknowledged
back to the producer.

Reviewers: Matthias J. Sax <matthias@confluent.io>

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
2025-06-16 18:40:54 -07:00
Nick Guo fd70290633
KAFKA-18486 Remove ReplicaManager#becomeLeaderOrFollower from testFencedErrorCausedByBecomeLeader and other similar methods (#19966)
CI / build (push) Waiting to run Details
The included tests are as follows:

- testFencedErrorCausedByBecomeLeader
- testFetchBeyondHighWatermark
- testFetchFollowerNotAllowedForOlderClients
- testFetchFromFollowerShouldNotRunPreferLeaderSelect
- testFetchFromLeaderAlwaysAllowed
- testFetchMessagesWhenNotFollowerForOnePartition
- testFetchRequestRateMetrics
- testFetchShouldReturnImmediatelyWhenPreferredReadReplicaIsDefined
- testFollowerFetchWithDefaultSelectorNoForcedHwPropagation
- testFollowerStateNotUpdatedIfLogReadFails

I removed `testFetchMessagesWithInconsistentTopicId ` as it's no longer
needed, the "topicId" is now mandatory and cannot be null in our new
implementation.

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Lan Ding
<isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-16 21:43:41 +08:00
Jhen-Yung Hsu 86419e9b8a
KAFKA-18486 Migrate ReplicaManagerTest#testTransactionAddPartitionRetry and other similar methods to use applyDelta (#19965)
CI / build (push) Waiting to run Details
Change becomeLeaderOrFollower to applyDelta in following test cases:

- testTransactionAddPartitionRetry
- testTransactionVerificationBlocksOutOfOrderSequence
- testTransactionVerificationDynamicDisablement
- testTransactionVerificationFlow
- testTransactionVerificationGuardOnMultiplePartitions
- testTransactionVerificationRejectsLowerProducerEpoch

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-16 03:48:36 +08:00
Chuckame a83bfda39b
MINOR: Fix typo to actually use rocksdb setOptimizeFiltersForHits (#19837)
CI / build (push) Waiting to run Details
This PR fixes a typo in the `RocksDBStore` where it currently uses a
getter named `optimizeFiltersForHits`. However, to set the flag to true,
we have to use `setOptimizeFiltersForHits(true)` instead.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-16 00:18:29 +08:00
Lan Ding 081deaa1a9
KAFKA-18486: Migrate ReplicaManagerTest to use applyDelta (#19954)
CI / build (push) Waiting to run Details
Change becomeLeaderOrFollower to applyDelta in following test cases

- testReadCommittedFetchLimitedAtLSO
- testReceiveOutOfOrderSequenceExceptionWithLogStartOffset
- testRemoteFetchExpiresPerSecMetric
- testRemoteLogReaderMetrics

Reviewers: PoAn Yang <payang@apache.org>, Bolin Lin
<linbolin1230@gmail.com>, Yung <yungyung7654321@gmail.com>, Jimmy Wang
<48462172+JimmyWang6@users.noreply.github.com>, Ken Huang
<s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-06-14 11:40:51 +08:00
Abhiram98 99e1e684ef
MINOR: rename exception `e` to `swallow` where appropriate (#19948)
CI / build (push) Waiting to run Details
I noticed that in commit

[a662bc56](a662bc5634),
renamed ignored exceptions `e` to `swallow`.

Here's a small patch to make that change consistent in other files:
AsyncKafkaConsumer.java, KafkaConsumerTest.java,
AsyncKafkaConsumerTest.java

Reviewers: Kirk True <kirk@kirktrue.pro>, Lan Ding <isDing_L@163.com>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-06-13 17:48:02 +08:00
Bolin Lin 94807bcd15
KAFKA-18486 Migrate ReplicaManagerTest RemoteFetchExpiresPerSecMetric and RemoteLogReaderMetrics with applyDelta (#19960)
CI / build (push) Waiting to run Details
Replace `becomeLeaderOrFollower` with `applyDelta` in method
RemoteFetchExpiresPerSecMetric and RemoteLogReaderMetrics

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-13 15:55:48 +08:00
PoAn Yang 991a10c19f
KAFKA-18486 Migrate ReplicaManagerTest#testOffsetOutOfRangeExceptionWhenFetchMessages to use applyDelta (#19952)
Change `becomeLeaderOrFollower` to `applyDelta` in following test cases
* testOffsetOutOfRangeExceptionWhenFetchMessages
* testOffsetOutOfRangeExceptionWhenReadFromLog
* testOldFollowerLosesMetricsWhenReassignPartitions
* testOldLeaderLosesMetricsWhenReassignPartitions

Reviewers: Bolin Lin <linbolin1230@gmail.com>, Lan Ding
 <isDing_L@163.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-13 15:47:51 +08:00
Kuan-Po Tseng 8c0d7412f4
KAFKA-18462: Upgrade RocksDB dependency from 9.7.3 to 10.1.3 (#19880)
Upgraded RocksDB from 9.7.3 to 10.1.3, deprecate two configuration in
`RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java`
- random_access_max_buffer_size (removed since v9.11.1

541761eaaa)
- rate_limiter (deprecated since v7.6.0

25cc564ff7)

Add one configuration:
- daily_offpeak_time_utc (introduced since v9.11.1

9b1d0c02e9)

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-06-13 09:37:37 +02:00
Kuan-Po Tseng 12d8a1bbf8
KAFKA-19237: Add dynamic config remote.log.manager.follower.thread.pool.size (#19809)
Deprecate the `remote.log.manager.thread.pool.size` configuration and introduce a new dynamic configuration:
`remote.log.manager.follower.thread.pool.size`.

Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Luke Chen <showuon@gmail.com>
2025-06-13 09:33:45 +05:30
Apoorv Mittal 2572c7ff9d
MINOR: Fixing client telemetry validate request (#19959)
CI / build (push) Waiting to run Details
Minor fix to correct the validate condition for GetTelemetryRequests.
Added respective tests as well.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-12 22:52:50 +01:00
Bill Bejeck 7e07659b7a
MINOR: Add comment why we use thread-id filtering when registering metrics for KIP-1076 (#19957)
Adding a descriptive comment why it's necessary to filter metrics
registration by thread-id tags.  This is due to the fact that the
`StreamsMetric` is a singleton shared by all StreamThread instances, so
we need to make sure only add metrics for the current StreamThread
otherwise duplicate metrics are registered.

Reviewers: Matthias Sax <mjsax@apache.org>
2025-06-12 17:20:52 -04:00
Kirk True 7d6e5edf8e
KAFKA-19153: Add OAuth integration tests (#19938)
Adds a test dependency on
[mock-oauth2-server](https://github.com/navikt/mock-oauth2-server/) for
integration tests for OAuth layer. Also includes fixes for some
regressions that were caught by the integration tests.

Reviewers: Manikumar Reddy <manikumar@confluent.io>, Lianet Magrans
 <lmagrans@confluent.io>
2025-06-12 15:48:14 -04:00
Ritika Reddy 0b2e410d61
KAFKA-19367: Fix InitProducerId with TV2 double-increments epoch if ongoing transaction is aborted (#19910)
CI / build (push) Waiting to run Details
When InitProducerId is handled on the transaction coordinator, the
producer epoch is incremented (so that we fence stale requests), then if
a transaction was ongoing during this time, it's aborted.  With
transaction version 2 (a.k.a. KIP-890 part 2), abort increments the
producer epoch again (it's the part of the new abort / commit protocol),
so the epoch ends up incremented twice.

In most cases, this is benign, but in the case where the epoch of the
ongoing transaction is 32766, it's incremented to 32767, which is the
maximum value for short. Then, when it's incremented for the second
time, it goes negative, causing an illegal argument exception.

To fix this we just avoid bumping the epoch a second time.

Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
 <alivshits@confluent.io>
2025-06-12 09:37:07 -07:00
Lianet Magrans 8f2ee4d7cd
KAFKA-18117; KAFKA-18729: Use assigned topic IDs to avoid full metadata requests on broker-side regex (#19814)
This PR uses topic IDs received in assignment (under new protocol) to
ensure that only these assigned topics are included in the consumer
metadata requests performed when the user subscribes to broker-side
regex (RE2J).

For handling the edge case of consumer needing metadata for topics IDs
(from RE2J) and topic names (from transient topics), the approach is to
send a request for the transient topics needed temporarily, and once
those resolved, the request for the topic IDs needed for RE2J will
follow. (this is because the broker doesn't accept requests for names
and IDs at the same time)

With the changes we also end up fixing another issue (KAFKA-18729) aimed
at avoiding iterating the full set of assigned partitions when checking
if a topic should be retained from the metadata response when using
RE2J.

Reviewers: David Jacot <djacot@confluent.io>
2025-06-12 10:50:31 -04:00
Kirk True 78c1da1aca
KAFKA-17089: Incorrect JWT parsing in OAuthBearerUnsecuredJws (#19946)
Fixed a long-standing issue where the client JWT validation was decoding
the JWT sections using base 64 instead of URL-safe base 64.

Note: server-side validation leverages the jose4j library for parsing
JWTs, hence no fix is needed there.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Manikumar Reddy
<manikumar@confluent.io>

---------

Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
2025-06-12 09:53:50 -04:00
Dongnuo Lyu af012e1ec2
KAFKA-18961: Time-based refresh for server-side RE2J regex (#19904)
Consumers can subscribe to an RE2J SubscriptionPattern that will be
resolved and maintained on the server-side (KIP-848). Currently, those
regexes are refreshed on the coordinator when a consumer subscribes to a
new regex, or if there is a new topic metadata image (to ensure regex
resolution stays up-to-date with existing topics)

But with
[KAFKA-18813](https://issues.apache.org/jira/browse/KAFKA-18813), the
topics matching a regex are filtered based on ACLs. This generates a new
situation, as regexes resolution do not stay up-to-date as topics become
visible (ACLs added/delete).

This patch introduces time-based refresh for the subscribed regex by
- Adding internal `group.consumer.regex.batch.refresh.max.interval.ms`
config
that controls the refresh interval.
- Schedule a regex refresh when updating regex subscription if the
latest refresh is older than the max interval.

Reviewers: David Jacot <djacot@confluent.io>
2025-06-12 04:54:39 -07:00
Jing-Jia Hung 2a7457f2dd
KAFKA-18486 Migrate testPartitionMetadataFile to use applyDelta in place of deprecated becomeLeaderOrFollower (#19947)
CI / build (push) Waiting to run Details
Refactor testPartitionMetadataFile to use applyDelta and share
class-level partitions

- Replace deprecated becomeLeaderOrFollower with topicsCreateDelta +
applyDelta
- Test still asserts partition exists, local log exists, and verifies
partitionMetadataFile version (0) and topicId

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-06-12 09:20:47 +08:00
Ken Huang 7c715c02c0
KAFKA-18486 Update testClearPurgatoryOnBecomingFollower etc with KRaft mechanism in ReplicaManagerTest (#19924)
CI / build (push) Waiting to run Details
update the following test to avoid using `becomeLeaderOrFollower`
- testClearPurgatoryOnBecomingFollower
- testDelayedFetchIncludesAbortedTransactions
- testDisabledTransactionVerification
- testFailedBuildRemoteLogAuxStateMetrics

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-06-11 18:52:08 +08:00
Nick Guo ab42f00bbe
KAFKA-18486 Remove ReplicaManager#becomeLeaderOrFollower in `testVerificationErrorConversions` (#19923)
Remove ReplicaManager#becomeLeaderOrFollower in
`testVerificationErrorConversionsTV1 ` and
`testVerificationErrorConversionsTV2 `.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 18:47:23 +08:00
Jhen-Yung Hsu 2e968560e0
MINOR: Cleanup simplify set initialization with Set.of (#19925)
Simplify Set initialization and reduce the overhead of creating extra
collections.

The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())

This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 18:36:14 +08:00
Mickael Maison b704280a0a
MINOR: Bump trunk to 4.2.0-SNAPSHOT (#19944)
CI / build (push) Waiting to run Details
Update the version as per
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=34840886#ReleaseProcess-CutBranches

Reviewers: Luke Chen <showuon@gmail.com>
2025-06-11 09:24:57 +02:00
Luke Chen ff58e90dfa
KAFKA-19359: force bump commons-beanutils for CVE-2025-48734 (#19939)
Bump the commons-beanutils for CVE-2025-48734. Since `commons-validator`
hasn't had new release with newer `commons-beanutils` versions, we manually bump it in kafka.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-06-11 15:23:04 +08:00
Chih-Yuan Chien 7dd07eb9ba
MINOR: Update opentelemetry-proto dependency (#19945)
CI / build (push) Waiting to run Details
Update opentelemetry-proto from 1.0.0-alpha to 1.3.2-alpha.

OpenTelemetry-Proto versions from v1.0.0 up to and including v1.3.2
introduce no breaking changes.

[release
note](https://github.com/open-telemetry/opentelemetry-proto/releases)

For example, starting with v1.4.0, protobuf-java was updated to version
4.28.3. To mitigate the risk of protobuf compatibility issues, upgrading
to v1.3.2 first allows the existing protobuf version to remain unchanged
for now.

Reviewers: poorv Mittal <apoorvmittal10@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 02:17:44 +08:00
Gaurav Narula edd0efdebf
KAFKA-19221 Propagate IOException on LogSegment#close (#19607)
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>
2025-06-11 01:09:52 +08:00
Apoorv Mittal 997abe464f
KAFKA-19389: Fix memory consumption for completed share fetch requests (#19928)
For ShareFetch Requests, the fetch happens through DelayedShareFetch
operation. The operations which are already completed has reference to
data being sent as response. As the operation is watched over multiple
keys i.e. DelayedShareFetchGroupKey and DelayedShareFetchPartitionKey,
hence if the operation is already completed by either  watched keys  but
then again the reference to the operation is still present in other
watched  key. Which means the memory can only be free once purge
operation is  triggered by DelayedOperationPurgatory which removes the
watched key  operation from remaining keys, as the operation is already
completed.

The purge operation is dependent on the config
`ShareGroupConfig#SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG`
hence if the value is not smaller than the number of share fetch
requests which can consume complete memory of the broker then broker can
go out of memory. This can also be avoided by having lower fetch max
bytes for request but this value is client dependent hence can't rely to
prevent  the broker.

This PR triggers the completion on both watched keys hence the
DelayedShareFetch operation shall be removed from both keys which frees
the broker memory as soon the share fetch response is sent.

#### Testing

Tested with LocalTieredStorage where broker goes OOM after reading some
8040 messages before the fix, with default configurations as mentioned
in the
doc

[here](https://kafka.apache.org/documentation/#tiered_storage_config_ex).
But after the fix the consumption continues without any issue. And the
memory is released instantaneously.

Reviewers: Jun Rao <junrao@gmail.com>, Andrew Schofield
<aschofield@confluent.io>
2025-06-10 17:36:27 +01:00
Dmitry Werner f69379cf6b
MINOR: Remove unused code from storage classes (#19853)
CI / build (push) Waiting to run Details
Remove unused code from storage classes.

Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>,
 TengYao Chi <kitingiao@gmail.com>, Kuan-Po Tseng <brandboat@gmail.com>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 00:22:50 +08:00
Alieh Saeedi 2b589a451a
KAFKA-19244: Add support for kafka-streams-groups.sh options (delete offsets) [4/N] (#19895)
This PR implements 
`--delete-offsets --all-input-topics`
`--delete-offsets --input-topic String: topicName`

Testing: integration test

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-10 14:55:48 +02:00
Ming-Yen Chung 472c2cfcb4
MINOR: Improve the prompt in committer-tools/reviewers.py (#19942)
Previously, the confirmation prompt for updating the PR body treated any
input other than 'n' as approval, which could lead to unintended
actions.

With this change, the update will only proceed if the user enters 'y',
'Y', or presses Enter. For any other input, the operation is canceled
and an Abort. message is printed. This makes the prompt behavior clearer
and more predictable.

Reviewers: TengYao Chi <frankvicky@apache.org>, PoAn Yang
<payang@apache.org>, Kuan-Po Tseng <brandboat@gmail.com>, Ken Huang
<s7133700@gmail.com>, Lan Ding <isDing_L@163.com>
2025-06-10 17:37:57 +08:00
Lucas Brutschy 793dcee541
KAFKA-19376: Throw an error message if any unsupported feature is used with KIP-1071 (#19908)
We should be mindful of ours users and let them know early if they are
using an unsupported feature in 4.1.

Unsupported features:

- Regular expressions
- Warm-up replicas (high availability assignor)
- Static membership
- Standby replicas enabled through local config
- Named topologies (already checked)
- Non-default kafka-client supplier

Reviewers: Bill Bejeck <bbejeck@apache.org>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-10 11:13:34 +02:00
Ming-Yen Chung 08aa469af7
KAFKA-19392 Fix metadata.log.segment.ms not being applied (#19936)
The original `props.setProperty(TopicConfig.SEGMENT_MS_CONFIG,
config.logSegmentMillis.toString)` in the `KafkaMetadataLog` constructor
was accidentally removed in #19371. Add a test to ensure this property
is properly  assigned.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-10 17:01:54 +08:00
Nick Guo 465b01cd2c
KAFKA-19382:Upgrade junit from 5.10 to 5.13 (#19919)
CI / build (push) Waiting to run Details
jira: https://issues.apache.org/jira/browse/KAFKA-19382

Upgrade junit from 5.10.2 to
[5.13.1](https://github.com/junit-team/junit5/releases).

A new behavior was introduced to junit 5.12

(89a46dfa10),
disallowing `ClusterTestExtensions` to generate empty invocation
contexts. However, `ClusterTestExtensions` is invoked by junit extension
so it could result in empty contexts for some tests.

```
> Configure project :
Starting build with version 4.1.0-SNAPSHOT (commit id c4a769bc) using
Gradle 8.14.1, Java 17 and Scala 2.13.16
Build properties: ignoreFailures=false, maxParallelForks=10,
maxScalacThreads=8, maxTestRetries=0

> Task :core:test kafka.api.ConsumerBounceTest.initializationError
failed, log available in
/Users/lansg/Project/OpenSource/kafka/kafka-fork/kafka/core/build/reports/testOutput/kafka.api.ConsumerBounceTest.initializationError.test.stdout

Gradle Test Run :core:test > Gradle Test Executor 5 > ConsumerBounceTest
> testCloseDuringRebalance(String) > initializationError FAILED
org.junit.platform.commons.PreconditionViolationException: Provider
[ClusterTestExtensions] did not provide any invocation contexts, but was
expected to do so. You may override
mayReturnZeroTestTemplateInvocationContexts() to allow this.         at
java.base@17.0.13/java.util.ArrayList.forEach(ArrayList.java:1511) at
java.base@17.0.13/java.util.ArrayList.forEach(ArrayList.java:1511)
kafka.api.ConsumerBounceTest.initializationError failed, log available
in
/Users/lansg/Project/OpenSource/kafka/kafka-fork/kafka/core/build/reports/testOutput/kafka.api.ConsumerBounceTest.initializationError.test.stdout

```

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>
2025-06-10 15:35:30 +08:00
Kuan-Po Tseng 3a0a1705a1
KAFKA-18486 Remove becomeLeaderOrFollower from readFromLogWithOffsetOutOfRange and other related methods. (#19929)
refactor out becomeLeaderOrFollower in below tests
- readFromLogWithOffsetOutOfRange
- testBecomeFollowerWhileNewClientFetchInPurgatory
- testBecomeFollowerWhileOldClientFetchInPurgatory
- testBuildRemoteLogAuxStateMetricsThrowsException

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
2025-06-10 12:39:32 +08:00
Kaushik Raina dbfda79951
KAFKA-19283: Update transaction exception handling documentation (#19931)
Added docs on Enhancements to transactional producer error handling:

* Added standardized exception categories (`RetriableException`,
`RefreshRetriableException`, `AbortableException`,
`ApplicationRecoverableException`, `InvalidConfigurationException`,
`KafkaException`) to ensure clearer error handling patterns.
* Included a link to example template code for handling transaction
exceptions: [Transaction Client

Demo](https://github.com/apache/kafka/blob/trunk/examples/src/main/java/kafka/examples/TransactionalClientDemo.java).

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-06-09 17:07:29 -07:00
Calvin Liu b420e4092e
MINOR: ELR release note for 4.1 (#19909)
CI / build (push) Waiting to run Details
Mention that ELR will be enabled by default on new clusters in 4.1

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-06-09 17:05:20 -07:00
Matthias J. Sax 0adc6fa3e1
KAFKA-19271: allow intercepting internal method call (#19832)
CI / build (push) Waiting to run Details
To allow intercepting the internal subscribe call to the async-consumer,
we need to extend ConsumerWrapper interface accordingly, instead of
returning the wrapped async-consumer back to the KS runtime.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-09 07:28:13 -07:00
YuChia Ma 948a91dfdf
MINOR: Safe update dependencies (#19897)
These dependencies have been updated across both files:

    caffeine: From 3.1.8 to 3.2.0      javassist: From 3.29.2-GA to
3.30.2-GA      Jetty-related: All Jetty components have been updated
from 12.0.15 to 12.0.22, including:          jetty-alpn-client
jetty-client          jetty-ee10-servlet          jetty-ee10-servlets
jetty-http          jetty-io          jetty-security
jetty-server          jetty-session          jetty-util      jose4j:
From 0.9.4 to 0.9.6      Jersey-related: All Jersey components have been
updated from 3.1.9 to 3.1.10, including:          jersey-client
jersey-common          jersey-container-servlet
jersey-container-servlet-core          jersey-hk2          jersey-server
classgraph: From 4.8.173 to 4.8.179      jline: From 3.25.1 to 3.30.4
pcollections: From 4.0.1 to 4.0.2      re2j: From 1.7 to 1.8
snappy-java: From 1.1.10.5 to 1.1.10.7

New Dependency (LICENSE-binary only)

    A new dependency, jspecify-1.0.0, has been added to LICENSE-binary.

gradle/dependencies.gradle Specific Updates

These updates are only reflected in the gradle/dependencies.gradle file:

    bcpkix: From 1.78.1 to 1.80      bndlib: From 7.0.0 to 7.1.0 jacoco:
From 0.8.10 to 0.8.13      hamcrest: From 2.2 to 3.0      jqwik: From
1.8.3 to 1.9.2

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-09 21:52:58 +08: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
Ken Huang d6861f3f15
MINOR: Use `pollUntilTrue` instead of `waitForCondition` (#19911)
CI / build (push) Waiting to run Details
We can use `pollUntilTrue` instead of `waitForCondition`, thus do a
little refactor to reduce the duplicate code

Reviewers: TengYao Chi <frankvicky@apache.org>, Lan Ding
 <isDing_L@163.com>, TaiJuWu <tjwu1217@gmail.com>
2025-06-09 15:33:00 +08: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
Ken Huang df73133f3b
MINOR: Follow up KAFKA-19080 MetadataLogConfig (#19842)
CI / build (push) Waiting to run Details
See Discussion:
https://github.com/apache/kafka/pull/19371#discussion_r2109549343

Do the following changes:
- Update the internal config name with metadata prefix
- add the warning message for setting
`INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG`

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-09 03:13:03 +08:00
Nick Guo e23c8cea07
KAFKA-18486 Remove ReplicaManager#becomeLeaderOrFollower from `testReplicaAlterLogDirs` (#19922)
Use `applyDelta` replace `becomeLeaderOrFollower`

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-06-09 03:00:00 +08:00
Ken Huang 8fd0d33670
KAFKA-19042 Move PlaintextConsumerSubscriptionTest to client-integration-tests module (#19827)
CI / build (push) Waiting to run Details
Use Java to rewrite PlaintextConsumerSubscriptionTest by new test infra
and move it to client-integration-tests module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-08 05:06:03 +08:00