Commit Graph

16374 Commits

Author SHA1 Message Date
Kevin Wu 857b1e92cc
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
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>
2025-09-25 12:56:16 -04:00
Lan Ding f4e00e9cf0
MINOR: Remove unnecessary check in ReplicaManager (#20588)
`setTopics` is executed at before, so the check is unnecessary.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-09-25 23:52:46 +08:00
Sanskar Jhajharia 97c8c6b595
KAFKA-19733 Fix arguments to assertEquals() in clients module (#20586)
The given PR mostly fixes the order of arguments in `assertEquals()` for
the Clients module. Some minor cleanups were included with the same too.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-25 23:37:02 +08:00
Jinhe Zhang 14917ae727
MINOR: Handle envelope response in AutoTopicCreationManager (#20569)
In the create topic request we send a CreateTopic request in an
Envelope, so we need to also unpack the response correctly

Reviewers: Lucas Brutschy <lucasbru@apache.org>
2025-09-25 11:06:22 +02:00
Mickael Maison 444ceeb325
MINOR: Tidy up the Connect docs (#20531)
Remove invalid mentions of default values for group.id,
config.storage.topic, offset.storage.topic, status.storage.topic

Reviewers: Luke Chen <showuon@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-09-25 09:39:37 +02:00
Maros Orsak 563338c0e9
MINOR: Refactor on DelegationTokenManager follow up with KAFKA-18711 (#20579)
Follow-up PR of KAFKA-18711. The motivation and reason for this change
are outlined in [1].

[1] - https://github.com/apache/kafka/pull/20475#discussion_r2375608168

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-25 02:25:44 +08:00
Lan Ding ac63ce9789
KAFKA-19544 Improve `MetadataVersion.fromVersionString()` to take an enableUnstableFeature flag (#20248)
Improve `MetadataVersion.fromVersionString()` to take an
`enableUnstableFeature` flag,   and enable `FeatureCommand` and
`StorageTool` to leverage the exception message from
`fromVersionString`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-25 01:06:54 +08:00
Shivsundar R 348e64c57d
MINOR: Add unit tests for verifying --formatter-property in console tools. (#20560)
*What*
In the implementation of KIP-1147 for console tools -

https://github.com/apache/kafka/pull/20479/files#diff-85b87c675a4b933e8e0e05c654d35d60e9cfd36cebe3331af825191b2cc688ee,
we missed adding unit tests for verifying the new
"`--formatter-property`" option.
Thanks to @Yunyung for pointing this out.

PR adds unit tests to both `ConsoleConsumerOptionsTest` and
`ConsoleShareConsumerOptionsTest` to verify the same.

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-09-25 00:15:05 +08:00
Dongnuo Lyu cbea4f69bd
KAFKA-19546: Rebalance should be triggered by subscription change during group protocol downgrade (#20417)
During online downgrade, when a static member using the consumer
protocol which is also the last member using the consumer protocol is
replaced by another static member using the classic protocol with the
same instance id, the latter will take the assignment of the former and
an online downgrade will be triggered.

In the current implementation, if the replacing static member has a
different subscription, no rebalance will be triggered when the
downgrade happens. The patch checks whether the static member has
changed subscription and triggers a rebalance when it does.

Reviewers: Sean Quah <squah@confluent.io>, David Jacot
 <djacot@confluent.io>
2025-09-24 15:40:45 +02:00
majialong 55020f909d
MINOR: Improve the documentation content for DistributedConfig (#20576)
1. Fix doc of `inter.worker.signature.algorithm` config in
`DistributedConfig`.
2. Improve the style of the `inter.worker.verification.algorithms` and
`worker.unsync.backoff.ms` config.
3. `INTER_WORKER_KEY_TTL_MS_MS_DOC` -> `INTER_WORKER_KEY_TTL_MS_DOC`.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-09-24 14:56:13 +02:00
Abhi Tiwari 1e4b8a1a6b
KAFKA-6333: java.awt.headless should not be on commandline (#20044)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-09-24 14:52:27 +02:00
Maros Orsak 486b991f22
KAFKA-18711 Move DelegationTokenPublisher to metadata module (#20475)
Basically, one of the refactor tasks. In this PR, I have moved
`DelegationTokenPublisher` to the metadata module. Similar to the
`ScramPublisher` migration (commit feee50f476), I have moved
`DelegationTokenManager` to the server-common module, as it would
otherwise create a circular dependency. Moreover, I have made multiple
changes throughout the codebase to reference `DelegationTokenManager`
from server-common instead of the server module.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-09-24 20:19:08 +08:00
Ken Huang 8036e49a6e
KAFKA-17554 Flaky testFutureCompletionOutsidePoll in ConsumerNetworkClientTest (#18298)
Jira: https://issues.apache.org/jira/browse/KAFKA-17554

In the previous workflow, the test passes under two conditions:

1. The `t1` thread is waiting for the main thread's `client.wakeup()`.
If successful, `t1` will wake up `t2`, allowing `t2` to complete the
future.
2. If `t1` fails to receive the `client.wakeup()` from the main thread,
`t2` will be woken up by the main thread.

In the previous implementation, we used a `CountDownLatch` to control
the execution of three threads, but it often led to race conditions.
Currently, we have modified it to use two threads to test this scenario.

I run `I=0;  while ./gradlew :clients:test --tests
ConsumerNetworkClientTest.testFutureCompletionOutsidePoll --rerun
--fail-fast; do  (( I=$I+1 )); echo "Completed run: $I"; sleep 1; done`
and pass 3000+ times.

![image](https://github.com/user-attachments/assets/3b8d804e-fbe0-4030-8686-4960fc717d07)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-24 18:42:25 +08:00
Lucas Brutschy 1f7631c8c6
MINOR: Fix StreamsRebalanceListenerInvoker (#20575)
StreamsRebalanceListenerInvoker was implemented to match the behavior of
ConsumerRebalanceListenerInvoker, however StreamsRebalanceListener has a
subtly different interface than ConsumerRebalanceListener - it does not
throw exceptions, but returns it as an Optional.

In the interest of consistency, this change fixes this mismatch by
changing the StreamsRebalanceListener interface to behave more like the
ConsumerRebalanceListener - throwing exceptions directly.

In another minor fix, the StreamsRebalanceListenerInvoker is changed to
simply skip callback execution instead of throwing an
IllegalStateException when no streamRebalanceListener is defined. This
can happen when the consumer is closed before Consumer.subscribe is
called.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Matthias J. Sax
 <matthias@confluent.io>
2025-09-24 09:03:07 +02:00
Ritika Reddy 0a483618b9
KAFKA-19690-Add epoch check before verification guard check to prevent unexpected fatal error (#20534)
We are seeing cases where a Kafka Streams (KS) thread stalls for ~20
seconds. During this stall, the broker correctly aborts the open
transaction (triggered by the 10-second transaction timeout).   However,
when the KS thread resumes, instead of receiving the expected
InvalidProducerEpochException (which we already handle gracefully as
part of transaction abort), the client is instead hit with an
InvalidTxnStateException. KS currently treats this as a fatal error,
causing the application to fail.

To fix this, we've added an epoch check before the verification check to
send the recoverable  InvalidProducerEpochException instead of the fatal
InvalidTxnStateException. This helps safeguard both tv1 and tv2 clients

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-09-23 13:45:42 -07:00
ally heev dbe9d34e47
KAFKA-19624: Improve consistency of command-line arguments for consumer performance tests (#20385)
resolves https://issues.apache.org/jira/browse/KAFKA-19624

Reviewers: @brandboat, @AndrewJSchofield, @m1a2st
2025-09-23 10:01:40 +01:00
Matthias J. Sax 71efb89290
MINOR: fix incorrect offset reset logging (#20558)
We need to only pass in the reset strategy, as the `logMessage`
parameter was removed.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Lucas Brutschy
 <lbrutschy@confluent.io>
2025-09-22 18:54:50 +02:00
Uladzislau Blok f16d1f3c9d
KAFKA-19299: Fix race condition in RemoteIndexCacheTest (#19927)
This MR should be couple of race conditions in RemoteIndexCacheTest.

1. There was a race condition between cache-cleanup-thread and test
thread, which wants to check that cache is gone. This was fixed with
TestUtils#waitForCondition
2. After each test we check that there is not thread leak. This check
wasn't working properly, because live of thread status is set by JVM
level, we can only set interrupted status (using private native void
interrupt0(); method under the hood), but we don't really know when JVM
will change the live status of thread. To fix this I've refactored
TestUtils#assertNoLeakedThreadsWithNameAndDaemonStatus method to use
TestUtils#waitForCondition. This fix should also affect few other tests,
which were flaky because of this check. See gradle run on

[develocity](https://develocity.apache.org/scans/tests?search.rootProjectNames=kafka&search.timeZoneId=Europe%2FLondon&tests.container=org.apache.kafka.storage.internals.log.RemoteIndexCacheTest&tests.sortField=FLAKY)

After fix test were run 10000 times with repeated test annotation:

`./gradlew clean storage:test --tests

org.apache.kafka.storage.internals.log.RemoteIndexCacheTest.testCacheEntryIsDeletedOnRemoval`
...  `Gradle Test Run :storage:test > Gradle Test Executor 20 >
RemoteIndexCacheTest > testCacheEntryIsDeletedOnRemoval() > repetition
9998 of 10000 PASSED`  `Gradle Test Run :storage:test > Gradle Test
Executor 20 > RemoteIndexCacheTest > testCacheEntryIsDeletedOnRemoval()
> repetition 9999 of 10000 PASSED`  `Gradle Test Run :storage:test >
Gradle Test Executor 20 > RemoteIndexCacheTest >
testCacheEntryIsDeletedOnRemoval() > repetition 10000 of 10000 PASSED`
`BUILD SUCCESSFUL in 20m 9s`  `148 actionable tasks: 148 executed`

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-09-22 11:20:14 -04:00
Ken Huang da6a562f6d
KAFKA-17834: Improvements to Dockerfile (#17554)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-22 17:04:42 +02:00
Ken Huang a0640f9517
KAFKA-18351: Remove top-level version field from docker-compose.yml files (#18322)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Sylwester Lachiewicz <slachiewicz@apache.org>, Dávid Szigecsán
2025-09-22 16:57:30 +02:00
Ken Huang 01fccd3513
KAFKA-15186 AppInfo metrics don't contain the client-id (#20493)
All Kafka component register AppInfo metrics to track the application
start time or commit-id etc. These metrics are useful for monitoring and
debugging. However, the AppInfo doesn't provide client-id, which is an
important information for custom metrics reporter.

The AppInfoParser class registers a JMX MBean with the provided
client-id, but when it adds metrics to the Metrics registry, the
client-id is not included. This KIP aims to add the client-id as a tag.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-21 16:28:03 +08:00
Ken Huang 07b786e5bf
KAFKA-19681 Improve MetadataShell tool by skipping missing children and removing zkMigrationState (#20504)
The current `metadata-shell` find command throws an exception due to
child node `zkMigrationState`.
This interrupts the output and makes the CLI less usable.

Additionally, `zkMigrationState` is no longer used in Kafka 4.x, but it
is still defined under image/features, which causes misleading error
messages.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-21 12:35:46 +08:00
Chang-Chi Hsu 5919762009
MINOR: Remove exitMessage.set() call in TopicBasedRemoteLogMetadataManagerTest (#20563)
- **Reasons:** In this case, the `exit(int statusCode)` method invokes
`exit(statusCode, null)`, which means
the `message` argument is always `null` in this code path. As a result,
assigning
`exitMessage` has no effect and can be safely removed.

- **Changes:** Remove a redundant field assignment.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-20 18:04:10 +08:00
Now c49ab6b4ae
MINOR: Optimize map lookup efficiency with getOrDefault (#20229)
Optimized `getRemainingRecords()` method by replacing inefficient
`containsKey() + get()` pattern with `getOrDefault()` to reduce map
lookups from 2 to 1 per partition.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-20 11:52:29 +08:00
Jhen-Yung Hsu 57e9f98e15
KAFKA-19644 Enhance the documentation for producer headers and integration tests (#20524)
- Improve the docs for Record Headers.
- Add integration tests to verify that the order of headers in a record
is preserved when producing and consuming.
- Add unit tests for RecordHeaders.java.

Reviewers: Ken Huang <s7133700@gmail.com>, Hong-Yi Chen
 <apalan60@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-20 11:46:19 +08:00
Lianet Magrans 848e3d0092
KAFKA-19722: Adding missing metric assigned-partitions for new consumer (#20557)
Adding the missing metric to track the number of partitions assigned.
This metric should be registered whenever the consumer is using a
groupId, and should track the number of partitions from the subscription
state, regardless of the subscription type (manual or automatic).

This PR registers the missing metric as part of the
ConsumerRebalanceMetricsManager setup. This manager is created if there
is a group ID, and reused by the consumer membershipMgr and the
streamsMemberhipMgr, so we ensure that the metric is registered for the
new consumer and streams.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, TengYao Chi
 <frankvicky@apache.org>
2025-09-19 12:42:43 -04:00
Lan Ding cfa0b416ef
MINOR: Remove metrics attribute from StreamsGroup (#20559)
The `metrics` attribute in `StreamsGroup` is not used anymore. This
patch removes it.

Reviewers: Ken Huang <s7133700@gmail.com>, Lucas Brutschy
 <lbrutschy@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-19 16:32:41 +08:00
Sean Quah d067c6c040
KAFKA-19716: Clear out coordinator snapshots periodically while loading (#20547)
When nested Timeline collections are created and discarded while loading
a coordinator partition, references to them accumulate in the current
snapshot. Allow the GC to reclaim them by starting a new snapshot and
discarding previous snapshots every 16,384 records.

Small intervals degrade loading times for non-transactional offset
commit workloads while large intervals degrade loading times for
transactional workloads. A default of 16,384 was chosen as a compromise.

Also add a benchmark for group coordinator loading.

Reviewers: David Jacot <djacot@confluent.io>
2025-09-19 09:44:07 +02:00
Ryan Dielhenn b72db2b2c7
MINOR: Delete temporary directories after using them in RaftManagerTest Updated (#20550)
Follow-up to [#11193](https://github.com/apache/kafka/pull/11193). This
change adds cleanup of the temporary log and metadata directories
created by RaftManagerTest so they are removed after each test run.
Without this cleanup, the directories remain until the entire test suite
completes, leaving extra files in the system temporary directory.

Testing:
- Ran `./gradlew core:test --tests kafka.raft.RaftManagerTest` and
confirmed all tests pass.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-09-19 10:22:05 +08:00
Andrew Schofield 5ed4a48829
MINOR: Tighten up argument descriptions for console CLI tools (#20554)
Small improvements to the argument descriptions in the usage messages
for the console producer/consumer tools.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-19 10:10:12 +08:00
Sean Quah dbd2b527d0
MINOR: Fix format in CoordinatorLoaderImplTest (#20548)
Fix indentation in `CoordinatorLoaderImplTest` to be consistent with the
rest of the code in the package.

Reviewers: TengYao Chi <kitingiao@gmail.com>, David Jacot <djacot@confluent.io>
2025-09-18 15:52:03 +02:00
David Jacot 8c8e93c4a1
MINOR: Remove metrics attribute from ConsumerGroup (#20542)
The `metrics` attribute in `ConsumerGroup` is not used anymore. This
patch removes it.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Dongnuo Lyu
 <dlyu@confluent.io>
2025-09-18 11:10:35 +02:00
David Jacot d6fdbfcf15
MINOR: Fix typos in CoordinatorRecordTypeGenerator (#20549)
This patch fixes a few typos in CoordinatorRecordTypeGenerator.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Sean Quah <squah@confluent.io>
2025-09-18 16:22:35 +08:00
Jinhe Zhang 04b4a8f571
KAFKA-19705: Enable streams rebalance protocol in IQv2 integration test (#20541)
Update IQv2 Integration tests for streams group protocol

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-09-18 09:41:52 +02:00
Shivsundar R e647bdcee5
MINOR : Fix parantheses in console_consumer.py and console_share_consumer.py (#20552)
*What*
We were missing a parantheses when we invoked a method
`supports_formatter_property()`. This would mean we would get the object
not call the function.
PR fixes this by including parantheses and invoking the actual function.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-09-18 11:53:12 +05:30
Lan Ding 9a32a71e76
KAFKA-19699 improve the documentation of `RecordsToDelete` (#20527)
document the behavior of "-1" (HIGH_WATERMARK)

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-17 23:49:42 +08:00
Shivsundar R 3bc50f937c
KAFKA-19623: Implement KIP-1147 for console producer/consumer/share-consumer. (#20479)
*What*
https://issues.apache.org/jira/browse/KAFKA-19623

- The PR implements KIP-1147

(https://cwiki.apache.org/confluence/display/KAFKA/KIP-1147%3A+Improve+consistency+of+command-line+arguments)
for the console tools i.e. `ConsoleProducer`, `ConsoleConsumer` and
`ConsoleShareConsumer`.

- Currently the previous names for the options are still usable but
there will be warning message stating those are deprecated and will be
removed in a future version.
- I have added unit tests and also manually verified using the console
tools that things are working as expected.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, Jimmy Wang
 <48462172+JimmyWang6@users.noreply.github.com>
2025-09-17 15:28:20 +01:00
David Jacot bbbc0cf793
MINOR: Fix format in CoordinatorLoaderImpl (#20538)
The format of the code in `CoordinatorLoaderImpl` in inconsistent with
the rest of the code in the package. This small PR fixes it.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Sean
 Quah <squah@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-17 17:13:28 +08:00
Jinhe Zhang 8ba41a2d0d
MINOR: Expose internal topic creation errors to the user (#20325)
This PR introduces an ExpiringErrorCache that temporarily stores topic
creation errors, allowing the system to provide detailed failure reasons
in subsequent heartbeat responses.

Key Designs:

Time-based expiration: Errors are cached with a TTL based on the
streams group heartbeat interval (2x heartbeat interval). This ensures
errors remain available for at least one retry cycle while preventing
unbounded growth.    2. Priority queue for efficient expiry: Uses a
min-heap to track entries by expiration time, enabling efficient cleanup
of expired entries during cache operations.    3. Capacity enforcement:
Limits cache size to prevent memory issues under high error rates. When
capacity is exceeded, oldest entries are evicted first.    4. Reference
equality checks: Uses eq for object identity comparison when cleaning up
stale entries, avoiding expensive value comparisons while correctly
handling entry updates.

Reviewers: Lucas Brutschy <lucasbru@apache.org>
2025-09-16 20:52:39 +02:00
Shashank b043ca2074
KAFKA-19683: Remove dead tests and modify tests in TaskManagerTest [1/N] (#20501)
This is the first part of cleaning up of the tests in `TaskManagerTest`
- Removed dead tests
- Added new tests as suggested earlier

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-09-16 20:46:20 +02:00
Lianet Magrans 9f657abf3a
MINOR: Improve consumer rebalance callbacks docs (#20528)
Clarify rebalance callbacks behaviour (got some questions for
onPartitionsAssigned, docs where indeed confusing about the partitions
received in params).   Reviewed all rebalance callbacks with it.

Reviewers: Bill Bejeck<bbejeck@apache.org>
2025-09-16 11:12:19 -04:00
Lucas Brutschy 2c347380b7
KAFKA-19694: Trigger StreamsRebalanceListener in Consumer.close (#20511)
In the consumer, we invoke the consumer rebalance onPartitionRevoked or
onPartitionLost callbacks, when the consumer closes. The point is that
the application may want to commit, or wipe the state if we are closing
unsuccessfully.

In the StreamsRebalanceListener, we did not implement this behavior,
which means when closing the consumer we may lose some progress, and in
the worst case also miss that we have to wipe our local state state
since we got fenced.

In this PR we implement StreamsRebalanceListenerInvoker, very similarly
to ConsumerRebalanceListenerInvoker and invoke it in Consumer.close.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Matthias J. Sax
 <matthias@confluent.io>, TengYao Chi <frankvicky@apache.org>,
 Uladzislau Blok <123193120+UladzislauBlok@users.noreply.github.com>

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
2025-09-16 16:32:47 +02:00
Lan Ding daa7aae0c1
KAFKA-19604 Document controller.quorum.auto.join.enable config in upgrade.html (#20409)
Document controller.quorum.auto.join.enable config in upgrade.html

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-16 16:34:14 +08:00
Jhen-Yung Hsu dddb619177
MINOR: Move RaftManager interface to raft module (#20366)
- Move the `RaftManager` interface to raft module, and remove the
`register` and `leaderAndEpoch` methods since they are already part of
the RaftClient APIs.
- Rename RaftManager.scala to KafkaRaftManager.scala.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-16 16:19:42 +08:00
Mickael Maison 3cbb2a0aaf
MINOR: Small cleanups in clients (#20530)
- Fix non-constant calls to logging
- Fix assertEquals order
- Fix javadoc

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-09-16 03:56:11 +08:00
Lianet Magrans caeca090b8
MINOR: Improve producer docs and add tests around timeout behaviour on missing topic/partition (#20533)
Clarify timeout errors received on send if the case is topic not in
metadata vs partition not in metadata.  Add integration tests showcases
the difference  Follow-up from 4.1 fix for misleading timeout error
message (https://issues.apache.org/jira/browse/KAFKA-8862)

Reviewers: TengYao Chi <frankvicky@apache.org>, Kuan-Po Tseng
 <brandboat@gmail.com>
2025-09-15 13:28:27 -04:00
Alex 3fcc0c2877
MINOR: Fix an off-by-one issue in ValuesTest (#20520)
This test case ensures that the parser can convert ISO8601 correctly.
However, when the local time falls on a different day than the UTC time,
there will be an off-by-one issue.

I changed the test to convert the local time and then compare it with
the expected local time. This should fix the off-by-one issue.

[Reference
link](https://github.com/apache/kafka/pull/18611#discussion_r2318146619)

Reviewers: Andrew Schofield <aschofield@confluent.io>

---------

Signed-off-by: Alex <wenhsuan.alexyu@gmail.com>
2025-09-15 18:26:47 +01:00
Lucas Brutschy 8628d74c49
KAFKA-19661 [6/N]: Use heaps also on the process-level (#20523)
In the current solution, we only use a heap to select the right process,
but resort to linear search for selecting a member within a process.
This means use cases where a lot of threads run within the same process
can yield slow assignment. The number of threads in a process shouldn’t
scale arbitrarily (our assumed case for benchmarking of 50 threads in a
single process seems quite extreme already), however, we can optimize
for this case to reduce the runtime further.

Other assignment algorithms assign directly on the member-level, but we
cannot do this in Kafka Streams, since we cannot assign tasks to
processes that already own the task. Defining a heap directly on members
would mean that we may have to skip through 10s of member before finding
one that does not belong to a process that does not yet own the member.

Instead, we can define a separate heap for each process, which keeps the
members of the process by load. We can only keep the heap as long as we
are only changing the load of the top-most member (which we usually do).
This means we keep track of a lot of heaps, but since heaps are backed
by arrays in Java, this should not result in extreme memory
inefficiencies.

In our worst-performing benchmark, this improves the runtime by ~2x on
top of the optimization above.

Also piggybacked are some minor optimizations / clean-ups:   -
initialize HashMaps and ArrayLists with the right capacity   - fix some
comments   - improve logging output

Note that this is a pure performance change, so there are no changes to
the unit tests.

Reviewers: Bill Bejeck<bbejeck@apache.org>
2025-09-15 17:19:53 +02:00
Hong-Yi Chen 749c2d91d5
KAFKA-19609 Move TransactionLogTest to transaction-coordinator module (#20460)
This PR migrates the `TransactionLogTest` from Scala to Java for better
consistency with the rest of the test suite and to simplify future
maintenance.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-09-15 11:25:54 +08:00
Yunchi Pang e1b7699975
MINOR: Bump versions in CI (#20497)
**Summary**

This PR bumps several GitHub Actions and dependencies used in CI
workflows to their latest stable versions. This ensures our CI
environment remains consistent, secure, and aligned with upstream
improvements.

**Changes**

- requests: 2.32.3 → 2.32.4
- actions/checkout: v4 → v5
- actions/setup-python: v5 → v6
- actions/setup-java: v4 → v5
- actions/download-artifact: v4 → v5
- actions/labeler: v5 → v6

related: https://github.com/apache/kafka/pull/19940/files#r2328391161

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-09-15 10:35:42 +08:00