Commit Graph

16083 Commits

Author SHA1 Message Date
Sanskar Jhajharia 56aeaa4c44
MINOR: Cleanup ShareFetchAcknowledgeRequestTest (#19852)
CI / build (push) Waiting to run Details
Now that Kafka supports Java 17, this PR cleans up the
ShareFetchAcknowledgeRequestTest.
The changes mostly include:
- Collections.singletonList() is replaced with List.of()
- Get rid of all asJava conversions

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-26 13:56:18 +08:00
Hong-Yi Chen dc82c766fa
KAFKA-18834 Fix LoggingResourceTest#testSetLevelDefaultScope (#19920)
- Flaky behavior
`LoggingResourceTest#testSetLevelDefaultScope` sometimes fails by not
capturing its expected WARN log.

- Root cause
Both `LoggersTest#testSetLevelWithValidRootLoggerNames` and
`LoggingResourceTest#testSetLevelDefaultScope` may share the same
`LoggerContext` when executed in the same JVM.
`LoggersTest#testSetLevelWithValidRootLoggerNames` calls
`loggers.setLevel("", ERROR)`, which mutates the global root logger
level to ERROR and suppresses WARN logs, which causes subsequent tests
to fail to emit WARN-level output.

- Fix in this PR
Resets the Log4j configuration after each test in `LoggersTest`,
ensuring that any global changes are reverted.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-26 13:39:08 +08:00
Mahsa Seifikar 7aaba96cc1
KAFKA-19282: Update quotaTypesEnabled on quota removal in ClientQuotaManager (#19742)
CI / build (push) Waiting to run Details
In `kafka.server.ClientQuotaManager` class, `quotaTypesEnabled` is not updated when a quota is removed via `removeQuota` method in `DefaultQuotaCallback` class. This field is set when quotas are added in `updateQuota` but it's never changed or cleared. So in case all the quotas have been removed dynamically, the system may incorrectly assume the quotas are active, which leads to unnecessary metric creation or updates until the broker is restarted.

Reviewers: Jonah Hooper <jhooper@confluent.io>, Hailey Ni <hni@confluent.io>, Alyssa Huang <ahuang@confluent.io>, David Jacot <djacot@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
2025-06-25 21:29:46 +01:00
Lucas Brutschy 23ddb1d8ac
MINOR: Reject requests using unsupported features in KIP-1071 (#20031)
CI / build (push) Waiting to run Details
KIP-1071 does not currently support all features planned in the KIP. We
should reject any requests that are using features that are currently
not implemented.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax
 <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>
2025-06-25 14:48:56 +02:00
Jing-Jia Hung 5e23df0c8d
KAFKA-18486 Migrate tests to use applyDelta instead of becomeLeaderOrFollower for testInconsistentIdReturnsError and others (#20014)
continues the migration effort for KAFKA-18486 by replacing usage of the
deprecated `becomeLeaderOrFollower` API with `applyDelta` in several
test cases.

#### Updated tests:
- `testInconsistentIdReturnsError`
- `testMaybeAddLogDirFetchers`
- `testMaybeAddLogDirFetchersPausingCleaning`
- `testSuccessfulBuildRemoteLogAuxStateMetrics`
- `testVerificationForTransactionalPartitionsOnly`
- `testBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate`

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-25 20:02:27 +08:00
Rajini Sivaram 33a1648c44
MINOR: Fix response for consumer group describe with empty group id (#20030)
ConsumerGroupDescribe with an empty group id returns a response containing `null` groupId in a non-nullable field. Since the response cannot be serialized, this results in UNKNOWN_SERVER_ERROR being returned to the client. This PR sets the group id in the response to an empty string instead and adds request tests for empty group id.

Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 10:33:44 +01:00
Yu-Syuan Jheng 2d943419a3
MINOR: Remove unused DescribeTopicPartitionResponse.prepareResponse function (#20032)
The prepareResponse function is no longer used anywhere in the codebase.
Removing unused code improves maintainability and readability.

Reviewers: PoAn Yang <payang@apache.org>, Yung
<yungyung7654321@gmail.com>, TengYao Chi <frankvicky@apache.org>, Ken
Huang <s7133700@gmail.com>
2025-06-25 17:26:22 +08:00
Matthias J. Sax 4387132926
KAFKA-19398: (De)Register oldest-iterator-open-since-ms metric dynamically (#20022)
CI / build (push) Waiting to run Details
The metric for oldest-iterator-open-since-ms might report a null value
if there is not open iterator.

This PR changes the behavior to dynamically register/deregister the
entire metric instead of allowing it to return a null value.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2025-06-24 17:01:36 -07:00
Colin Patrick McCabe 6b2013a001
KAFKA-19294: Fix BrokerLifecycleManager RPC timeouts (#19745)
CI / build (push) Waiting to run Details
Previously, we could wait for up to half of the broker session timeout
for an RPC to complete, and then delay by up to half of the broker
session timeout. When taken together, these two delays could lead to
brokers erroneously missing heartbeats.

This change removes exponential backoff for heartbeats sent from the
broker to the controller. The load caused by heartbeats is not heavy,
and controllers can easily time out heartbeats when the queue length is
too long. Additionally, we now set the maximum RPC time to the length of
the broker period. This minimizes the impact of heavy load.

Reviewers: José Armando García Sancio <jsancio@apache.org>, David Arthur <mumrah@gmail.com>
2025-06-24 16:23:25 -07:00
Calvin Liu dcc9320bf6
KAFKA-19383: Handle the deleted topics when applying ClearElrRecord (#19916)
https://issues.apache.org/jira/browse/KAFKA-19383  When applying the
ClearElrRecord, it may pick up the topicId in the image without checking
if the topic has been deleted. This can cause the creation of a new
TopicRecord with an old topic ID.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-06-24 13:26:35 -07:00
Lan Ding 51f2c7b2b6
MINOR: fix reassign command bug (#20003)
see

9570c67b8c/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala (L1208)
During the rewrite for
[KAFKA-14595](https://github.com/apache/kafka/pull/13247), the relevant
condition was omitted.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 02:34:13 +08:00
Alyssa Huang 3460ded386
KAFKA-19411: Fix deleteAcls bug which allows more deletions than max records per user op (#19974)
If there are more deletion filters after we initially hit the
`MAX_RECORDS_PER_USER_OP` bound, we will add an additional deletion
record ontop of that for each additional filter.

The current error message returned to the client is not useful either,
adding logic so client doesn't just get `UNKNOWN_SERVER_EXCEPTION` with
no details returned.
2025-06-24 11:24:50 -07:00
Ken Huang 023833fe1f
KAFKA-18778 Fix the inconsistent lastest supported version in StorageTool.scala and FutureCommand (#19157)
To maintain code consistency, `MetadataVersion#fromVersionString` uses
`latestTesting()` as the latest version. Therefore, in the tools, we
also need to maintain consistency by updating the outer logging to use
`latestTesting()`.

See the discussion:
https://github.com/apache/kafka/pull/18845#discussion_r1950706791

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 01:59:52 +08:00
Okada Haruki 959021de59
KAFKA-19407 Fix potential IllegalStateException when appending to timeIndex (#19972)
## Summary
- Fix potential race condition in
LogSegment#readMaxTimestampAndOffsetSoFar(), which may result in
non-monotonic offsets and causes replication to stop.
- See https://issues.apache.org/jira/browse/KAFKA-19407 for the details
how it happen.

Reviewers: Vincent PÉRICART <mauhiz@gmail.com>, Jun Rao
 <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-25 00:35:53 +08:00
Alieh Saeedi d61b162b13
KAFKA-19244: Fix flaky streams offsets reset integration test (#20027)
CI / build (push) Waiting to run Details
This PR deflakes the
`testResetOffsetsWithDeleteSpecifiedInternalTopics()`

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-06-24 14:45:39 +02:00
Apoorv Mittal 1ca8779bee
MINOR: Correcting client error for fenced share partition (#20023)
Correct the error when SharePartition is fenced.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Sushant Mahajan
 <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-06-24 09:46:14 +01:00
Sushant Mahajan 3d4407ff9d
MINOR: Change exceptions for few error codes in SharePartition. (#20020)
CI / build (push) Waiting to run Details
* The `SharePartition` class wraps the errors received from
`PersisterStateManager` to be sent to the client.
* In this PR, we are categorizing the errors a bit better.
* Some exception messages in `PersisterStateManager` have been updated
to show the share partition key.
* Tests have been updated wherever needed.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal
 <apoorvmittal10@gmail.com>
2025-06-23 19:27:15 +01:00
Lucas Brutschy 4fedffd282
KAFKA-19429: Deflake streams_smoke_test (#20019)
It looks like we are checking for properties that are not guaranteed
under at_least_once, for example, exact counting (not allowing for
overcounting).

This change relaxes the validation constraint to only check that we
counted _at least_ N messages, and our sums come out as _at least_ the
expected sum.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-23 20:16:52 +02:00
Lucas Brutschy 261e861340
KAFKA-19422: Deflake streams_application_upgrade_test (#20004)
In this upgrade test, applications sometimes crash before the upgrade,
so it's actually triggering a bug in several older versions (2.x and
possibly others). It seems to be a rare race condition that has been
happening since 2022. Since we are not going to roll out a patch release
for Kafka Streams 2.x, we should just allow applications to crash before
the upgrade.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-23 17:41:05 +02:00
Sushant Mahajan cb809e2574
MINOR: Change snapshot epoch type to int32 in schema. (#20016)
CI / build (push) Waiting to run Details
* `SnapshotEpoch` type in `ShareSnapshotValue.json` and
`ShareUpdateValue.json` is currently
`uint16` which might overflow under heavy traffic.
* To be consistent with other epochs, this PR updates the type to
`int32`.

Reviewers: Andrew Schofield <aschofield@confluent.io>, ShivsundarR
 <shr@confluent.io>
2025-06-23 14:15:01 +01:00
Ming-Yen Chung b38573fcaa
KAFKA-18486 Remove becomeLeaderOrFollower from testPartition*, testPreferredReplicaAs* (#20009)
Replace `leaderAndIsrRequest` and `becomeLeaderOrFollower` with
`TopicsDelta`, `MetadataImage` and `ReplicaManager#applyDelta` for the
following tests:
* testPartitionListener
* testPartitionMarkedOfflineIfLogCantBeCreated
* testPartitionMetadataFileNotCreated
* testPartitionsWithLateTransactionsCount
* testPreferredReplicaAsFollower
* testPreferredReplicaAsLeader
* testPreferredReplicaAsLeaderWhenSameRackFollowerIsOutOfIsr
* testProducerIdCountMetrics

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-23 16:42:30 +08:00
Lucas Brutschy 4377ee0808
KAFKA-19421: Deflake streams_broker_down_resilience_test (#19999)
`streams_broker_down_resilience_test` produce messages with `null` key
to a topic with three partitions and expect each partition to be
non-empty afterward. But I don't think this is a correct assumption, as
a producer may try to be sticky and only produce to two partitions.

This cause occasional flakiness in the test.

The fix is to produce records with keys.

Reviewers: Matthias J. Sax <matthias@confluent.io>, PoAn Yang
 <payang@apache.org>
2025-06-23 10:30:13 +02:00
Sushant Mahajan 56a6ba2d2e
MINOR: Add retention prop to share group state topic. (#20013)
CI / build (push) Waiting to run Details
*

https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka
states the `retention.ms` property for the `__share_group_state` to be
`-1`.
* This PR makes it explicit when defining the values of those configs.
* Existing test has been updated.

```
$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --describe
--topic __share_group_state

Topic: __share_group_state      TopicId: XCwzZjEGSjm5lUc_BeCrqA
PartitionCount: 50      ReplicationFactor: 1
Configs:
compression.type=producer,
min.insync.replicas=1,
cleanup.policy=delete,
segment.bytes=104857600,
retention.ms=-1
...
```

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-22 22:02:35 +01:00
Bolin Lin 3404f65cdb
KAFKA-19324 Make org.apache.kafka.common.test.TestUtils package-private to prevent cross-module access (#19884)
Description

* Replace `org.apache.kafka.common.test.TestUtils` with
`org.apache.kafka.test.TestUtils` in outer package modules to
standardize test utility usage
* Move `waitUntilLeaderIsElectedOrChangedWithAdmin` method from
`org.apache.kafka.test.TestUtils` to `ClusterInstance` and refactor for
better code organization
* Add `org.apache.kafka.test.TestUtils` dependency to
`transaction-coordinator` import control

Reviewers: PoAn Yang [payang@apache.org](mailto:payang@apache.org), Ken
 Huang  [s7133700@gmail.com](mailto:s7133700@gmail.com), Ken Huang
 [s7133700@gmail.com](mailto:s7133700@gmail.com), Chia-Ping Tsai
 [chia7712@gmail.com](mailto:chia7712@gmail.com)
2025-06-22 22:47:40 +08:00
Nick Guo 583acb60d6
MINOR: Restore original behavior of GroupAuthorizerIntegrationTest (#19968)
CI / build (push) Waiting to run Details
this is a follow-up for https://github.com/apache/kafka/pull/19685

The timeout issue in `AsyncConsumer#unsubscribe` was fixed by
https://github.com/apache/kafka/pull/19779. As a result, the test

`GroupAuthorizerIntegrationTest#testConsumeUnsubscribeWithoutGroupPermission`
should now retain its original behavior as expected prior to the issue.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-22 22:08:20 +08:00
S.Y. Wang 22bef988d4
KAFKA-18926 KafkaPrincipalBuilder should extend KafkaPrincipalSerde (#19987)
In KRaft, custom KafkaPrincipalBuilder instances must implement
KafkaPrincipalSerde to support the forward mechanism. Currently, this
requirement is not enforced and relies on the developer's attention.

With this patch, we can prevent incorrect implementations at compile
time.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-22 22:01:03 +08:00
Ken Huang fa4d8836d3
MINOR: DescribeAuthorizedOperationsTest cluster setting has been overridden (#20012)
The old approach `OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG` will override
`OFFSETS_TOPIC_PARTITIONS_CONFIG` config, this behaviour is not
expected, we should fix it.

Reviewers: TengYao Chi <frankvicky@apache.org>
2025-06-22 21:42:03 +08:00
Sanskar Jhajharia 992eaafb62
MINOR: Cleanup Core Module- Scala Modules (3/n) (#19804)
CI / build (push) Waiting to run Details
Now that Kafka Brokers support Java 17, this PR makes some changes in
core module. The changes in this PR are limited to only some Scala files
in the Core module's tests. 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()

To be clear, the directories being targeted in this PR from unit.kafka
module:
- admin
- cluster
- coordinator
- docker
- integration
- metrics

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-22 00:20:39 +08:00
Chirag Wadhwa 7c77519f59
MINOR: changed the test testShareFetchRequestSuccessfulSharingBetweenMultipleConsumers to remove ambiguity (#19997)
CI / build (push) Waiting to run Details
The test testShareFetchRequestSuccessfulSharingBetweenMultipleConsumers
was recently found to be flaky. Making the following small change that
could potentially resolve the issue. Earlier, 1000 records were being
produced and then 3 consecutive share fetch requests were being sent. At
the end, assertions were done to make sure each share consumer receives
some records, and that none of them consume the same record. Since the
motive for the test is to see if multiple consumers can share the same
subscription and not consume the same record, a better way would be to
produce a record, consume that and repeat it 3 times with the 3
consumers. This ensures that every consumer consume a record, and a
previously consume record is not consumed again by the subsequent share
fetches.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield
 <aschofield@confluent.io>
2025-06-21 08:36:31 +01:00
Sushant Mahajan 815dd93e2f
MINOR: Invoke share group rebalance sensor. (#20006)
* The share group rebalance metric was not being invoked at the
appropriate group id bump position.
* This PR solves the issue.
* The metric name has been updated
(s/rebalance-rate/share-group-rebalance-rate,
s/rebalance-count/share-group-rebalance-count/)
* Updated tests in `GroupMetadataManagerTest` and
`GroupCoordinatorMetricsTest`

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-21 08:35:19 +01:00
Mason Chen d442c31e92
KAFKA-19402: Typo in EventAccumulator.java (#19951)
CI / build (push) Waiting to run Details
Fix multiple typo and grammar issues in EventAccumulator.java

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Lianet Magrans
 <lmagrans@confluent.io>

---------

Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
2025-06-20 15:34:02 -04:00
Lucas Brutschy 88a73c35e7
KAFKA-19423: Deflake streams_broker_bounce_test (#20005)
The test is resizing the `__consumer_offset` topic after broker start.
This seems to be completely unsupported. The group coordinator fetches
the number of partitions for the consumer offset topic once and never
updates it. So we can be in a state where two brokers have a different
understanding of how `__consumer_offsets` are partitioned.

The result in this test can be that two group coordinators both think
they own a certain group. The test is resizing `__consumer_offsets`
right after start-up from 3 to 50. Before the broker bounce, the GC
operates on only three partitions (0-2). During the bounce, we get new
brokers that operate on (0-49). This means that two brokers can both
think, at the same time, that they own a group.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-20 11:27:35 -07:00
Andrew Schofield 4690527fab
KAFKA-19362: Finalize homogeneous simple share assignor (#19977)
Finalise the share group SimpleAssignor for homogeneous subscriptions.
The assignor code is much more accurate about the number of partitions
assigned to each member, and the number of members assigned for each
partition. It eliminates the idea of hash-based assignment because that
has been shown to the unhelpful. The revised code is very much more
effective at assigning evenly as the number of members grows and shrinks
over time.

A future PR will address the code for heterogeneous subscriptions.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-06-20 16:10:47 +01:00
Alieh Saeedi a797d644bc
KAFKA-19244: Add support for kafka-streams-groups.sh options (reset-offsets) [1/N] (#19646)
This PR implements all the options for `--reset-offset`, (supports
`--execute`, `--dry-run`, `--export`). Includes unit and integration
tests.

- --reset-offsets --input-topic String: topics --shift-by Long
- --reset-offsets --input-topic String: topics --to-offset  Long
- --reset-offsets --input-topic String: topics -to-latest
- --reset-offsets --input-topic String: topics -to-earliest
- --reset-offsets --all-input-topics --shift-by Long
- --reset-offsets --all-input-topics --to-offset  Long
- --reset-offsets --all-input-topics -to-latest
- --reset-offsets --all-input-topics -to-earliest
- --reset-offsets --all-input-topics -to-earliest
- --reset-offsets --from-file String:fileName
- --reset-offsets --input-topic String: topics --by-duration String:
duration
- --reset-offsets --all-input-topics --by-duration String: duration
- --reset-offsets --input-topics <String: topics> --to-datetime String:
datetime
- --reset-offsets --all-input-topics --to-datetime String: datetime
- --reset-offsets --input-topics <String: topics> -to-current
- --reset-offsets --all-input-topics -to-current

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Bill Bejeck
 <bill@confluent.io>
2025-06-20 16:14:18 +02:00
Lianet Magrans 80eb86e18e
KAFKA-19418: Fix for flaky verifiable producer system test (#20001)
Fix to avoid flakiness in verifiable producer system test. The test
lists running processes and greps to find the VerifiableProducer one,
but wasn't providing an specific pattern to grep (so flaky if there were
more than one process containing the default grep pattern "kafka")

Fix by passing a "proc_grep_string"  to filter when looking for the
VerifiableProducer process.

All test pass successfully after the change.

Reviewers: PoAn Yang <payang@apache.org>, Andrew Schofield
<aschofield@confluent.io>
2025-06-20 09:59:10 -04:00
Sushant Mahajan d5e2ecae95
MINOR: Reduce logging in persister. (#19998)
CI / build (push) Waiting to run Details
* Few logs in `PersisterStateManager` were noisy and not adding much
value.
* For the sake of reducing pollution, they have been moved to debug
level.
* Additional debug log in `DefaultStatePersister` to track epochs.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield
 <aschofield@confluent.io>
2025-06-20 13:53:46 +01:00
Lan Ding 659ace836c
MINOR: Replace `singleton` with `Set.of` in the SubscriptionStateTest (#19993)
Replace `singleton` with `Set.of` in the SubscriptionStateTest.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Yung <yungyung7654321@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-20 15:33:41 +08:00
Juha Mynttinen d59d39a229
MINOR: fix MetadataRequest and MetadataResponse doc (#19963)
CI / build (push) Waiting to run Details
MetadataRequest and MetadataResponse version 0 is still supported.
Remove from README.md mentioning that they're not supported. It's
unclear how such a mention was ever included in the file. It has always
been there

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-19 22:06:02 +01:00
Ken Huang 15ad3016b7
KAFKA-19140 ConnectAssignor#performAssignment parameter can be replace to ConnectProtocolCompatibility (#19476)
CI / build (push) Waiting to run Details
The protocol type; for Connect assignors this is "eager", "compatible",
or "sessioned"

Since `ConnectAssignor` is an interface and the protocol parameter is
restricted to "eager", "compatible", or "sessioned", it aligns with the
existing ConnectProtocolCompatibility enum. Therefore, we can update the
code to use `ConnectProtocolCompatibility` directly.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-20 03:07:57 +08:00
Ming-Yen Chung 2c3ce72a05
KAFKA-18903 update-cache.sh use curl + jq to get sha contains gradle cache instead of gh cli (#19753)
Instead of using the GH CLI (which requires some local setup), we can
just use cURL and JQ. Additionally, the original logic required setting
up alias.update-cache. After the change, this configuration is no longer
necessary.

validation on mac:

```
❯ ./update-cache.sh
Local branch 'trunk-cached' successfully updated to f26974b16d (from 19
hours ago).
```

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-20 03:02:17 +08:00
Chang-Chi Hsu 46b474a9de
KAFKA-19239 Rewrite IntegrationTestUtils by java (#19776)
This PR rewrites the IntegrationTestUtils.java from Scala to Java.

## Changes:

- Converted all the existing Scala code in IntegrationTestUtils.scala
into Java in IntegrationTestUtils.java.
- Preserved the original logic and functionality to ensure backward
compatibility.
- Updated relevant imports and dependencies accordingly.

Motivation:

The rewrite aims to standardize the codebase in Java, which aligns
better with the rest of the project and facilitates easier maintenance
by the Java-centric team.

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>,
 PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-20 01:46:29 +08:00
Lan Ding dbbc45f7a0
KAFKA-19416: Reduce running time of ShareConsumerTest (#19992)
Configure tests in ShareConsumer to run exclusively in Raft-Isolated
mode to reduce execution time.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-06-19 15:30:23 +01:00
Shivsundar R 55297967a9
MINOR : Fix possible flake in ShareConsumerTest:testConsumerCloseInGroupSequential (#19986)
CI / build (push) Waiting to run Details
*What*

- There was a possibility of a flake occurring in one of the tests in
`ShareConsumerTest` where we are not waiting for the assignment to be
received and synced before we start consuming on `poll()`.

- There should ideally be a `waitedPoll()` or a dedicated poll which
would trigger the reconcilation and updation of the assignment.

Scenario :

- We are doing 3 `polls` to receive 1500 records.

- On debugging, I noticed 1st poll always returned 0 records as this was
triggering reconcilation, the test passed locally as 2nd and 3rd polls
add up to give us 1500 records almost always (as 500 is a soft limit).
The test can fail if these 2 `polls` do not add up 1500. We need a
maximum
of 3 polls to receive 1500 records with `max.poll.records` set to 500.
- Now we have introduced a loop which would keep polling until we
receive the expected records and then close.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-19 12:53:49 +01:00
Xuan-Zhang Gong 79d2c3c62a
KAFKA-19406 Remove BrokerTopicStats#removeOldFollowerMetrics (#19962)
BTW: whether we should rename
`ReplicaManager#updateLeaderAndFollowerMetrics`

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, TengYao Chi <kitingiao@gmail.com>, Lan Ding
 <isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-19 17:57:22 +08:00
Ken Huang 297bc40d96
MINOR: The upgrade.html file contains duplicate IDs on the same page (#19996)
According to correct HTML syntax, IDs on the same page should be unique,
so we should fix this.

Reviewers: TengYao Chi <frankvicky@apache.org>
2025-06-19 17:18:58 +08:00
Kuan-Po Tseng 812ddee2b3
MINOR: document deprecated remote log manager thread pool config (#19989)
Add a note to the upgrade documentation indicating that
`remote.log.manager.thread.pool.size` has been deprecated. Users should
now use `remote.log.manager.follower.thread.pool.size` instead.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
 <chia7712@gmail.com>, Yung <yungyung7654321@gmail.com>, Xuan-Zhang Gong
 <gongxuanzhangmelt@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Ken
 Huang <s7133700@gmail.com>
2025-06-19 17:17:50 +08:00
Logan Zhu 772b7c2a5f
MINOR: Add Docker version requirement and permission warning to README (#19994)
Adds a new section **"Installation Preparation"** to the Kafka Docker
examples README. It documents a known issue with Docker versions prior
to 20.10.4, where directory permissions (such as `/opt/kafka/config`)
are not properly set, causing startup failures for non-root users (e.g.,
`appuser`).

This issue was fixed in Docker [20.10.4
release](https://docs.docker.com/engine/release-notes/20.10/#20104),
specifically in Use 0755 permissions when creating missing directories
[moby/moby#42017](https://github.com/moby/moby/pull/42017).

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Lan Ding
 <isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>

---------

Co-authored-by: Jhen-Yung Hsu <yungyung7654321@gmail.com>
2025-06-19 17:07:45 +08:00
Lianet Magrans ce996b34e9
KAFKA-19356: Prevent new consumer fetch assigned partitions not in explicit subscription (#19983)
CI / build (push) Waiting to run Details
Fix to ensure assigned partitions whose topics are not in the consumer
explicit subscription are considered not fetchable (so that no records
are returned on poll for them)

This scenario could happen in the new async consumer (using the Consumer
rebalance protocol) when the subscription changes, because the consumer
will keep its assignment until the coordinator sends a new one (broker
drives assignments).

This does not happen in the classic consumer because the assignment
logic lives on the client-side, so the consumer pro-actively updates
assignment as needed.

This PR validates assignment vs subscription on fetch for explicit
subscription  only. Regular expressions, shared subscription  remain
unchanged (regex case still under discussion, will be handled separately
if needed)

Reviewers: Andrew Schofield <aschofield@confluent.io>, TengYao Chi
 <frankvicky@apache.org>, Kirk True <ktrue@confluent.io>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>
2025-06-18 18:31:46 -04:00
Xuan-Zhang Gong b8fc96272e
KAFKA-18627 Deprecated "org.apache.kafka.disallowed.login.modules" (#18683)
CI / build (push) Waiting to run Details
Fix https://issues.apache.org/jira/browse/KAFKA-18627 and update same
test case


[KIP-link](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=340037077)

⚠️ This PR cannot be merged at the moment because KIPs for version 4.1
are already frozen. It will need to wait for version 4.2.

update upgrade.html


![image](https://github.com/user-attachments/assets/00bd6579-6914-42a7-a1b8-4a0b0a8cc3c3)

add configuration doc

![image](https://github.com/user-attachments/assets/e20d5c97-29c6-4fe4-a41a-58267179e4e0)

Reviewers: TaiJuWu <tjwu1217@gmail.com>, Ken Huang <s7133700@gmail.com>,
 Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-18 23:05:30 +08:00
Kirk True adcf10ca8b
KAFKA-19297: Refactor AsyncKafkaConsumer's use of Java Streams APIs in critical sections (#19917)
Profiling has shown that using the Collections Streams API approach adds
unnecessary overhead compared to a traditional for loop. Minor revisions
to the code have been made to use simpler constructs to improve
performance.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-06-18 10:00:45 -04:00