Commit Graph

5659 Commits

Author SHA1 Message Date
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
Azhar Ahmed 832dfa36da
KAFKA-18637: Fix max connections per ip and override reconfigurations (#19099)
Reviewers: Christo Lolov <lolovc@amazon.com>, TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-03-10 07:27:48 +00:00
Ken Huang d5413fdb48
KAFKA-17856 Move ConfigCommandTest and ConfigCommandIntegrationTest to tool module (#17767)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 21:05:36 +08:00
PoAn Yang a5e5e2dcd5
KAFKA-18706 Move AclPublisher to metadata module (#18802)
Move AclPublisher to org.apache.kafka.metadata.publisher package.

Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 21:00:33 +08:00
ClarkChen 1584d49470
KAFKA-18944 Remove unused setters from ClusterConfig (#19166)
Remove unused `saslServerProperties`, `saslClientProperties`,
`adminClientProperties`, `producerProperties`, and `consumerProperties`
in ClusterConfig.

First, I quickly fixed the unused adminClientProperties, and then I will
move on to https://github.com/apache/kafka/pull/19094 to fix the related
issues.

Pass AdminClientRebootstrapTest
<img width="1398" alt="Screenshot 2025-03-09 at 12 54 57 PM"
src="https://github.com/user-attachments/assets/73c50376-6602-493d-8abd-0eb2bb304114"
/>

Pass ClusterConfigTest
<img width="1117" alt="Screenshot 2025-03-09 at 12 55 28 PM"
src="https://github.com/user-attachments/assets/b4da59da-dfdf-4698-9077-5086854360ab"
/>

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 17:49:28 +08:00
ClarkChen 2a0dbd8e0b
KAFKA-18909 Move DynamicThreadPool to server module (#19081)
* Add `DynamicThreadPool.java` to the server module.
* Remove the old DynamicThreadPool object in the `DynamicBrokerConfig.scala`.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 17:42:51 +08: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
Dániel Urbán 40db001588
KAFKA-18929: Log a warning when time based segment delete is blocked by a future timestamp (#19137)
When producers send future timestamps, time retention based log segments
may get blocked from removal for an extended period of time. Log
cleaning should should warn in the logs when this scenario occurs.

Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2025-03-07 14:31:22 +01:00
ClarkChen 870db5d811
KAFKA-18915: Migrate AdminClientRebootstrapTest to use new test infra (#19094)
Migrate AdminClientRebootstrapTest to the new test infra and remove the
old Scala test.

Reviewers: TengYao Chi <kitingiao@gmail.com>, David Arthur <mumrah@gmail.com>
2025-03-06 16:05:51 -05:00
Andrew Schofield 1da30bdedf
KAFKA-18900: Experimental share consumer acknowledge mode config (#19113)
User testing of the `KafkaShareConsumer` interface has revealed some
areas which confuse people. One of these is that way that it decides
whether you want to use implicit or explicit acknowledgement of records
by observing which calls the application issues. We are taking the
opportunity to refine the interface before it is finalised.

This PR introduces an experimental configuration called
`internal.share.acknowledgement.mode` which can be used to make the
application declare which kind of acknowledgement it wishes to use. We
plan to try out the configuration, assess whether it has helped, and
then create a proper consumer configuration that makes this area better.
That would require a lot of change in the tests, which explains why this
initial PR only has a small number of tests.

Reviewers: David Arthur <mumrah@gmail.com>
2025-03-06 17:57:11 +00:00
Ken Huang 041d8019d6
KAFKA-18910 Remove kafka.utils.json (#19112)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-06 14:11:20 +08: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
Xuan-Zhang Gong 18eca0229d
KAFKA-18882 Remove BaseKey, TxnKey, and UnknownKey (#19054)
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-05 21:16:18 +08:00
Lan Ding 69ff5d1e70
KAFKA-18817: ShareGroupHeartbeat and ShareGroupDescribe API must check topic describe (#19083)
Reviewers: Christo Lolov <lolovc@amazon.com>
2025-03-05 11:25:08 +00:00
Kuan-Po Tseng cbd72cc216
KAFKA-14121: AlterPartitionReassignments API should allow callers to specify the option of preserving the replication factor (#18983)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi <kitingiao@gmail.com>
2025-03-05 11:23:12 +00:00
Logan Zhu 011f256c86
KAFKA-18886 add behavior change of CreateTopicPolicy and AlterConfigPolicy to zk2kraft (#19087)
1. Updated JavaDoc to reflect that CreateTopicPolicy and AlterConfigPolicy run on the controller in KRaft mode.
2. Modified Behavioral Change Reference in the HTML docs to include this change.
3. add warning message to KafkaConfig if the config of broker node has policy configs 


Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-05 15:15:03 +08:00
co63oc e4ece37dbf
Fix typos in multiple files (#19086)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-04 16:05:51 +00:00
Apoorv Mittal c1fc59fc23
KAFKA-18918: Correcting releasing of locks on exception (#19091)
The PR corrects the way the locks are released on exception. As
`partitionsAcquired` can be a reference to `topicPartitionData`, hence
the locks should released prior clearing `partitionsAcquired`.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-03-04 16:04:45 +00:00
David Jacot 1df4a42b40
KAFKA-18916; Resolved regular expressions must update the group by topics data structure (#19088)
When regular expressions are resolved, they do not update the group by
topics data structure. Hence, topic changes (e.g. deletion) do not
trigger a rebalance of the group.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-03-04 06:31:08 -08:00
Nick Guo 101e15bb1c
KAFKA-18867 add tests to describe topic configs with empty name (#19075)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-04 14:56:25 +08:00
Mahsa Seifikar 2154e55abf
MINOR: Prevent broker fencing by adjusting resendExponentialBackoff in BrokerLifecycleManager (#19061)
This PR reduces `maxInterval` for `resendExponentialBackoff` in
`BrokerLifecycleManager` class from `broker.session.timeout.ms` to half
of its value. Setting `maxInterval` to `broker.session.timeout.ms`
caused brokers to be fenced if a resend attempt occurred near the
timeout threshold, leading to unnecessary broker fencing.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-03-03 12:03:15 -08:00
Apoorv Mittal a6c53d0c37
KAFKA-18878: Added share session cache and delayed share fetch metrics (KIP-1103) (#19059)
The PR implements the ShareSessionCache and DelayedShareFetchMetrics as
defined in KIP-1103.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-03 16:44:34 +00:00
Lucas Brutschy a04dd21f26
KAFKA-18613: Auto-creation of internal topics in streams group heartbeat (#18981)
Implements auto-topic creation when handling the streams group
heartbeat.

Inside KafkaApis, the handler for streamsGroupHeartbeat uses the result
of the streams group heartbeat inside the group coordinator to attempt
to create all missing internal topics using AutoTopicCreationManager.
CREATE TOPIC ACLs are checked. The unit tests class
AutoTopicCreationManagerTest is brought back (it was recently deleted
during a ZK removal PR), but testing only the kraft-related
functionality.

Reviewers: Bruno Cadonna <bruno@confluent.io>

### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation 
- [ ] Verify test coverage and CI build status
- [ ] Verify documentation (including upgrade notes)
2025-03-03 08:48:00 +01:00
Xuan-Zhang Gong ceac4f0a1d
KAFKA-18880 Remove kafka.cluster.Broker and BrokerEndPointNotAvailableException (#19047)
Remove kafka.cluster.Broker and BrokerEndPointNotAvailableException as they were used by zk path.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-02 10:54:32 +08:00
TengYao Chi e0c77140b2
KAFKA-17039 KIP-919 supports for unregisterBroker (#19063)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 23:55:35 +08:00
Nick Guo 98bb79e732
KAFKA-17981 add Integration test for ConfigCommand to add config `key=[val1,val2]` (#17771)
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 13:15:25 +08:00
Apoorv Mittal 8cf969e00a
KAFKA-18734: Implemented share partition metrics (KIP-1103) (#19045)
The PR implements the SharePartitionMetrics as defined in KIP-1103, with
one change. The metric `FetchLockRatio` is defined as `Meter` in KIP but
is implemented as `HIstogram`. There was a discussion about same on
KIP-1103 discussion where we thought that `FetchLockRatio` is
pre-aggregated but while implemeting the rate from `Meter` can go above
100 as `Meter` defines rate per time period. Hence it makes more sense
to implement metric `FetchLockRatio` as `Histogram`.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-28 14:22:27 +00:00
Apoorv Mittal 8b605bd362
MINOR: Removing share partition manager flaky annotation (#19053)
There isn't any flaky test for SharePartitionManager in last 7 days, removing flaky annotation.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-28 08:49:59 +00:00
Dongnuo Lyu 36f19057e1
KAFKA-18813: ConsumerGroupHeartbeat API and ConsumerGroupDescribe API must check topic describe (#18989)
This patch filters out the topic describe unauthorized topics from the
ConsumerGroupHeartbeat and ConsumerGroupDescribe response.

In ConsumerGroupHeartbeat, 
- if the request has `subscribedTopicNames` set, we directly check the
authz in `KafkaApis` and return a topic auth failure in the response if
any of the topics is denied.
- Otherwise, we check the authz only if a regex refresh is triggered and
we do it based on the acl of the consumer that triggered the refresh. If
any of the topic is denied, we filter it out from the resolved
subscription.

In ConsumerGroupDescribe, we check the authz of the coordinator
response. If any of the topic in the group is denied, we remove the
described info and add a topic auth failure to the described group.
(similar to the group auth failure)

Reviewers: David Jacot <djacot@confluent.io>, Lianet Magrans
<lmagrans@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>,
Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu <tjwu1217@gmail.com>,
TengYao Chi <kitingiao@gmail.com>
2025-02-26 13:05:36 -05:00
Lucas Brutschy cb7c54ccd3
KAFKA-18614, KAFKA-18613: Add streams group request plumbing (#18979)
This change implements the basic RPC handling StreamsGroupHeartbeat and
StreamsGroupDescribe. This includes:
 - Adding an option to enable streams groups on the broker
- Passing describe and heartbeats to the right shard of the group
coordinator
- The handler inside the GroupMetadatManager for StreamsGroupDescribe is
fairly trivial, and is included directly in this PR.
- The handler for StreamsGroupHeartbeat is complex and not included in
this PR yet. Instead, a UnsupportedOperationException is thrown.
However, the interface is already defined: The result of a
streamsGroupHeartbeat is a response, together with a list of internal
topics to be created.

The heartbeat implementation inside the `GroupMetadataManager`, which
actually implements the assignment / reconciliation logic, will come in
a follow-up PR. Also, automatic creation of internal topics will be
created in a follow-up PR.

Reviewers: Bill Bejeck <bill@confluent.io>
2025-02-26 16:33:26 +01:00
Abhinav Dixit 4b5a16bf6f
KAFKA-18757: Create full-function SimpleAssignor to match KIP-932 description (#18864)
### About
The current `SimpleAssignor` in AK assigned all subscribed topic
partitions to all the share group members. This does not match the
description given in
[KIP-932](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255070434#KIP932:QueuesforKafka-TheSimpleAssignor).
Here are the rules as mentioned in the KIP by which the assignment
should happen. We have changed the step 3 implementation here due to the
reasons
[described](https://github.com/apache/kafka/pull/18864#issuecomment-2659266502)
-

1. The assignor hashes the member IDs of the members and maps the
partitions assigned to the members based on the hash. This gives
approximately even balance.
2. If any partitions were not assigned any members by (1) and do not
have members already assigned in the current assignment, members are
assigned round-robin until each partition has at least one member
assigned to it.
3. We combine the current and new assignment. (Original rule - If any
partitions were assigned members by (1) and also have members in the
current assignment assigned by (2), the members assigned by (2) are
removed.)

### Tests
The added code has been verified with unit tests and the already present
integration tests.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, TaiJuWu <tjwu1217@gmail.com>
2025-02-26 11:02:23 +00:00
José Armando García Sancio 4a8a0637e0
KAFKA-18723; Better handle invalid records during replication (#18852)
For the KRaft implementation there is a race between the network thread,
which read bytes in the log segments, and the KRaft driver thread, which
truncates the log and appends records to the log. This race can cause
the network thread to send corrupted records or inconsistent records.
The corrupted records case is handle by catching and logging the
CorruptRecordException. The inconsistent records case is handle by only
appending record batches who's partition leader epoch is less than or
equal to the fetching replica's epoch and the epoch didn't change
between the request and response.

For the ISR implementation there is also a race between the network
thread and the replica fetcher thread, which truncates the log and
appends records to the log. This race can cause the network thread send
corrupted records or inconsistent records. The replica fetcher thread
already handles the corrupted record case. The inconsistent records case
is handle by only appending record batches who's partition leader epoch
is less than or equal to the leader epoch in the FETCH request.

Reviewers: Jun Rao <junrao@apache.org>, Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@apache.org>
2025-02-25 20:09:19 -05:00
Apoorv Mittal df5839a9f4
KAFKA-17351: Improved handling of compacted topics in share partition (2/N) (#19010)
The PR handles fetch for `compacted` topics. The fix was required only
when complete batch disappears from the topic log, and same batch is
marked re-available in Share Partition state cache. Subsequent log reads
will not result the disappeared batch in read response hence respective
batch will be left as available in the state cache.

The PR checks for the first fetched/read batch base offset and if it's
greater than the position from where the read occurred (fetch offset)
then if there exists any `available` batches in the state cache then
they will be archived.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-02-25 14:11:39 +00:00
xijiu 1edc30bf30
KAFKA-17836 Move RackAwareTest to server module (#19021)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 18:15:34 +08:00
TaiJuWu 1c82b89b4c
KAFKA-18712 Move Endpoint to server module (#18803)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Mickael Maison <mickael.maison@gmail.com>, Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 14:02:51 +08:00
PoAn Yang 10873e4210
KAFKA-18281: Kafka is improperly validating non-advertised listeners for routable controller addresses (#18387)
When a cluster is configured with a dynamic controller quorum, KRaft replica's endpoint are computed using the advertised.listeners property and not the quorum.controller.voters property. This change in the configuration makes it difficult to keeping all previous node configurations compatible with the new endpoint discovery functionality.

The least intrusive solution is to rely on Kafka's reverse hostname lookup when the hostname is not specified. The effective advertised controller listener now remove '0.0.0.0' hostname if the endpoint came from the listener configuration and not the advertised.listener configuration.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
2025-02-24 21:51:28 -05:00
Nick Guo d23a61738a
KAFKA-17937 Cleanup AbstractFetcherThreadTest (#18900)
- Remove AbstractFetcherThreadWithIbp26Test as it tests unsupported IBP
- cleanup AbstractFetcherThreadTest to remove unreachable paths, variables, and code

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-25 07:45:47 +08:00
Apoorv Mittal 48a506b7b8
KAFKA-18522: Slice records for share fetch (#18804)
The PR handles slicing of fetched records based on acquire response for
share fetch. There could be additional bytes fetched from log but
acquired offsets can be a subset, typically with `max fetch records`
configuration. Rather sending additional bytes of fetched data to client
we should slice the file and wire only needed batches.

Note: If the acquired offsets are within a batch then we need to send
the entire batch within the file record. Hence rather checking for
individual batches, PR finds the first and last acquired offset, and
trims the file for all batches between (inclusive) these two offsets.

Reviewers: Christo Lolov <lolovc@amazon.com>, Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2025-02-24 09:55:24 -08:00
mingdaoy 289e958c39
MINOR: Fix validateResourceNameIsNodeId's exception message (#19017)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-24 09:30:02 +08:00
Ismael Juma 13cb87c2d0
MINOR: Remove request log space added inadvertently (#19011)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-23 11:30:19 +08:00
Apoorv Mittal 6e45ab7d84
KAFKA-17351: Update tests and acquire API to allow discard batches from compacted topics (1/N) (#18978)
The PR does following:
1.  Adds `fetchOffset` to `acquire` API in SharePartition.
2. Adds a ShareFetchPartitionData class efficiently handle the
propagation of fetchOffset information.
3. Updates SharePartitionTests to make common code so such improvements
does not require all tests changes for future PRs.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-22 16:14:09 +00:00
Sushant Mahajan 4f28973bd1
KAFKA-18827: Initialize share state, share coordinator impl. [1/N] (#18968)
In this PR, we have added the share coordinator and KafkaApis side impl
of the intialize share group state RPC.
ref:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka#KIP932:QueuesforKafka-InitializeShareGroupStateAPI

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-22 16:12:08 +00:00
Apoorv Mittal f543eac4fe
KAFKA-18733: Implemented fetch ratio and partition acquire time metrics (3/N) (#18959)
PR implements the final set of ShareGroupMetrics,
RequestTopicPartitionsFetchRatio and TopicPartitionsAcquireTimeMs, as
defined in KIP-1103:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1103%3A+Additional+metrics+for+cooperative+consumption

Note: Metric `RequestTopicPartitionsFetchRatio` is calculated as
percentage as Histogram API doesn't record double.


Reviewers: Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-02-21 17:01:39 +00:00
Calvin Liu 8f13e7c207
MINOR: Move the ELR default version to 4.1 (#18954)
- ELR is enabled (ELRV_1) by default if the cluster is created with its bootstrap metadata version >= IBP_4_1_IV0.
- ELRV_1 can be manually enabled iff the metadata version is >= IBP_4_0_IV1.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>, David Jacot <djacot@confluent.io>
2025-02-21 16:13:11 +01:00
Shivsundar R 7da1a6cbff
KAFKA-18033: Remove flaky tag in ShareConsumerTest (#18995)
3 tests which were marked flaky in ShareConsumerTest do not have any
failure on trunk since the test was converted to use `ClusterTestExtensions`.

Reviewers: Sushant Mahajan <smahajan@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-21 13:50:08 +00:00
TengYao Chi 767a62ade6
KAFKA-18737 KafkaDockerWrapper setup functions fails due to storage format command (#18844)
The current Docker Hub documentation for Kafka is based on the use of static voters. Since Kafka 4.0 utilizes dynamic voters, users following the doc of docker hub may encounter unexpected behavior. Due to the limited time available for the 4.0.0 release, a simple and quick solution is to revert to using static voters within the Docker image. This can be achieved by adding a configuration file with static voter definitions to the kafka/docker folder, keeping it separate from the main kafka/config directory. This approach allows us to encourage the use of dynamic voters in typical deployments while maintaining compatibility within the Docker image.

Reviewers: Vedarth Sharma <142404391+VedarthConfluent@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 20:43:41 +08:00
TengYao Chi d31cbf59de
KAFKA-18831 Migrating to log4j2 introduce behavior changes of adjusting level dynamically (#18969)
fix the following behavior changes.

1) in log4j 1, users can't change the logger by parent if the logger is declared by properties explicitly. For example, `org.apache.kafka.controller` has level explicitly in the properties. Hence, we can't use "org.apache.kafka=INFO" to change the level of `org.apache.kafka.controller` to INFO. By contrast, log4j2 allows us to change all child loggers by the parent logger.

2) in log4j2, we can change the level of root to impact all loggers' level. By contrast, log4j 1 can't. 

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-21 16:12:58 +08:00
Calvin Liu 1eecd02ce8
MINOR: Deflake EligibleLeaderReplicasIntegrationTest (#18923)
Make sure to give enough time for the partition ISR updates.

Reviewers: David Jacot <djacot@confluent.io>
2025-02-20 05:14:15 -08:00
Matthias J. Sax 538a60e1b3
MINOR: disallow rawtypes and fail build (#18877)
Cleanup code to avoid rawtype, and add suppressions where necessary.
Change the build to fail on rawtype warning.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-19 13:11:49 -08:00
Ismael Juma 3a59a526d9
MIINOR: Remove redundant quorum parameter from *AdminIntegrationTest classes (#18965)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-02-19 15:57:47 -05:00