[0b2e410d61](url)
Bug fix in 4.0
**Conflicts:**
- The Transaction Coordinator had some conflicts, mainly with the
transaction states. Ex: ongoing in 4.0 is TransactionState.ONGOING in
4.1.
- The TransactionCoordinatorTest file had conflicts w.r.t the 2PC
changes from KIP-939 in 4.1 and the above mentioned state changes
Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
<alivshits@confluent.io>
Group Coordinator Shards are not unloaded when `__consumer_offsets`
topic is deleted. The unloading is scheduled but it is ignored because
the epoch is equal to the current epoch:
```
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1]
Scheduling unloading of metadata for __consumer_offsets-0 with epoch
OptionalInt[0]
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Scheduling
unloading of metadata for __consumer_offsets-1 with epoch OptionalInt[0]
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Ignored unloading
metadata for __consumer_offsets-0 in epoch OptionalInt[0] since current
epoch is 0.
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
[2025-05-13 08:46:00,883] INFO [GroupCoordinator id=1] Ignored unloading
metadata for __consumer_offsets-1 in epoch OptionalInt[0] since current
epoch is 0.
(org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime)
```
This patch fixes the issue by not setting the leader epoch in this case.
The coordinator expects the leader epoch to be incremented when the
resignation code is called. When the topic is deleted, the epoch is not
incremented. Therefore, we must not use it. Note that this is aligned
with deleted partitions are handled too.
Reviewers: Dongnuo Lyu <dlyu@confluent.io>, José Armando García Sancio <jsancio@apache.org>
The remote storage reader thread pool use same count for both maximum
and core size. If users adjust the pool size larger than original value,
it throws `IllegalArgumentException`. Updated both value to fix the
issue.
cherry-pick PR: #19532
cherry-pick commit:
965743c35b
---------
Signed-off-by: PoAn Yang <payang@apache.org>
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang
<payang@apache.org>
Co-authored-by: PoAn Yang <payang@apache.org>
Old bootstrap.metadata files cause problems with server that include
KAFKA-18601. When the server tries to read the bootstrap.checkpoint
file, it will fail if the metadata.version is older than 3.3-IV3
(feature level 7). This causes problems when these clusters are
upgraded.
This PR makes it possible to represent older MVs in BootstrapMetadata
objects without causing an exception. An exception is thrown only if we
attempt to access the BootstrapMetadata. This ensures that only the code
path in which we start with an empty metadata log checks that the
metadata version is 7 or newer.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma
<ismael@juma.me.uk>, PoAn Yang <payang@apache.org>, Liu Zeyu
<zeyu.luke@gmail.com>, Alyssa Huang <ahuang@confluent.io>
[KAFKA-18813](https://issues.apache.org/jira/browse/KAFKA-18813) added
`Topic:Describe` authorization of topics matching regex patterns to the
group coordinator since it was difficult to authorize these in the
broker when processing consumer heartbeats using the new protocol. But
group coordinator is started in `BrokerServer` before the authorizer is
created. And hence group coordinator doesn't have an authorizer and
never performs authorization. As a result, topics that are not
authorized for `Describe` may be assigned to consumers. This potentially
leaks information about topic existence, topic id and partition count to
users who are not authorized to describe a topic. This PR starts
authorizer earlier to ensure that authorization is performed by the
group coordinator. Also adds integration tests for verification.
Note that we still have a second issue when members have different
permissions. If regex is resolved by a member with permission to more
topics, unauthorized topics may be assigned to members with lower
permissions. In this case, we still return assignment containing topic
id and partitions to the member without `Topic:Describe` access. This is
not addressed by this PR, but an integration test that illustrates the
issue has been added so that we can verify when the issue is fixed.
Reviewers: David Jacot <david.jacot@gmail.com>
As of 3.9, Kafka allows disabling remote storage on a topic after it was
enabled. It allows subsequent enabling and disabling too.
However the documentation says otherwise and needs to be corrected.
Doc:
https://kafka.apache.org/39/documentation/#topicconfigs_remote.storage.enable
Reviewers: Luke Chen <showuon@gmail.com>, PoAn Yang <payang@apache.org>, Ken Huang <s7133700@gmail.com>
This PR fixes a potential issue where the `FetchResponse` returns
`divergingEndOffsets` with an older leader epoch. This can lead to
committed records being removed from the follower's log, potentially
causing data loss.
In detail:
`processFetchRequest` gets the requested leader epoch of partition data
by `topicPartition` and compares it with the leader epoch of the current
fetch state. If they don't match, the response is ignored.
Reviewers: Jun Rao <junrao@gmail.com>
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>
Backports f24945b519 to 4.0
Instead of reopening the transaction index, it cancels the RemoteFetchTask without interrupting it--avoiding to close the TransactionIndex channel.
This will lead to complete the execution of the remote fetch but ignoring the results. Given that this is considered a rare case, we could live with this. If it becomes a performance issue, it could be optimized.
Reviewers: Jun Rao <junrao@gmail.com>
Currently, each log.append() will add at most 1 index entry, even when
the appended data is larger than log.index.interval.bytes. One potential
issue is that if a follower restarts after being down for a long time,
it may fetch data much bigger than log.index.interval.bytes at a time.
This means that fewer index entries are created, which can increase the
fetch time from the consumers.
(cherry picked from commit e124d3975b)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
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>
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>
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>
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>
… 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)
(cherry picked from commit 36f19057e1)
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>
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>
- 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>
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>
Cherry-picked
3603c8fe35
into 4.0.
This was a bug fix to address
https://issues.apache.org/jira/browse/KAFKA-18829.
Now, we will only move to IMPLICIT mode in `ShareConsumerImpl`, if there
were any records to be acknowledged, and if the next
`poll()`/`commitAsync()`/`commitSync()` was called.
Reviewers: Andrew Schofield <aschofield@confluent.io>
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>
3.3.0 was the first KRaft release that was deemed production-ready and also
when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable
for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also
needs to be set to "3.3" or newer before upgrading).
Noteworthy changes:
1. `AlterPartition` no longer includes topic names, which makes it possible to
simplify `AlterParitionManager` logic.
2. Metadata versions older than `IBP_3_3_IV3` have been removed and
`IBP_3_3_IV3` is now the minimum version.
3. `MINIMUM_BOOTSTRAP_VERSION` has been removed.
4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`,
`isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and
`isInControlledShutdownStateSupported` - these are always `true` now.
Also removed related conditional code.
5. Removed default metadata version or metadata version fallbacks in
multiple places - we now fail-fast instead of potentially using an incorrect
metadata version.
6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord`
based on whether image is empty - this was a previously existing issue that
became more apparent after the changes in this PR.
7. Remove `ibp` parameter from `BootstrapDirectory`
8. A number of tests were not useful anymore and have been removed.
I will update the upgrade notes via a separate PR as there are a few things that
need changing and it would be easier to do so that way.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
- update reflection-config.json and resource-config.json to include log4j2 and jackson
- remove unused jackson scala library
- fix the incorrect path of log4j2.yaml
- adopt workaround (--standalone) to make this PR work and it will be fixed by KAFKA-18737)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
In PR #18267, we removed old message format for cases in ConsumerWithLegacyMessageFormatIntegrationTest. Although test cases can pass, they don't fulfill original purpose. We can't send old message format since 4.0, so I change cases to append old records by ReplicaManager directly.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Frequently updating the trust store can cause unexpected termination of the AsyncConsumer background thread.
1. To resolve this issue, reuse the same AdminClient instead of recreating it.
2. Add error logging when fail to initialize resources for the consumer network thread.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
During the transaction commit phase, it is normal to hit CONCURRENT_TRANSACTION error before the transaction markers are fully propagated. Instead of letting the client to retry the produce request, it is better to retry on the server side.
Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
The case testDescribeQuorumRequestToControllers shutdowns raft client but not the controller. This makes client has chance to send a request to the controller and get NOT_LEADER_OR_FOLLOWER error. However, if the raft client finishes shutdown before handling the request, the request will not be handled. Shutdown the controller before doing KafkaFuture#get for the client request, so we can make sure the request is handled by another controller eventually.
Signed-off-by: PoAn Yang <payang@apache.org>
Reviewers: Luke Chen <showuon@gmail.com>
* KAFKA-18758: NullPointerException in shutdown following InvalidConfigurationException
Add checks for null in shutdown as BrokerLifecycleManager is not instantiaited if LogManager constructor throws an Exception
After log4j migration, we need to update the logging configuration in KafkaDockerWrapper from log4j1 to log4j2.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
This commit ensures that the ClientQuotaCallback#updateClusterMetadata method is executed in KRaft mode. This method is triggered whenever a topic or cluster metadata change occurs. However, in KRaft mode, the current implementation of the updateClusterMetadata API is inefficient due to the requirement of creating a full Cluster object. To address this, a follow-up issue (KAFKA-18239) has been created to explore more efficient mechanisms for providing cluster information to the ClientQuotaCallback without incurring the overhead of a full Cluster object creation.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Remove broker.id.generation.enable and reserved.broker.max.id, which are not used in KRaft mode.
Remove inter.broker.protocol.version, which is not used in KRaft mode.
Reviewers: PoAn Yang <payang@apache.org>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
Since zk has been removed in 4.0, config handlers no longer need to handle the "<default>" value. This PR streamlines the config update process by eliminating the unnecessary string checks for "<default>"
Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
related to KAFKA-18206, set features in EmbeddedKafkaCluster in both streams and connect module, note that this PR also fix potential transaction with empty records in sendPrivileged method as transaction version 2 doesn't allow this kind of scenario.
Reviewers: Justine Olshan <jolshan@confluent.io>
This change implement some of the metrics enumerated in KIP-853.
The KafkaRaftMetrics object now exposes number-of-voters, number-of-observers and uncommitted-voter-change. The number-of-observers and uncommitted-voter-change metrics are only present on the active controller or leader, since it does not make sense for other replicas to report these metrics.
In order to make these two metrics thread-safe, KafkaRaftMetrics needs to be passed into LeaderState, and therefore QuorumState. This introduces a circularity since the KafkaRaftMetrics constructor takes in QuorumState. To break the circularity for now, the logic using QuorumState will be moved to the KafkaRaftMetrics#initialize method.
The BrokerServerMetrics object now exposes ignored-static-voters. The ControllerServerMetrics object now exposes IgnoredStaticVoters. To implement both metrics for "ignored static voters", this PR introduces the ExternalKRaftMetrics interface, which allows for higher layer metrics objects to be accessible within the raft module.
Reviewers: José Armando García Sancio <jsancio@apache.org>
We need to re-enable the unclean shutdown detection when in ELR mode, which was inadvertently removed during the development process.
Reviewers: David Mao <dmao@confluent.io>, Jun Rao <junrao@gmail.com>
There are times when the controller has a high event processing time, such as during startup, or when creating a topic with many partitions. We can see these processing times in the p99 metric (kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs), however it's difficult to see exactly which event is causing high processing time.
With DEBUG logs, we see every event along with its processing time. Even with this, it's a bit tedious to find the event with a high processing time.
This PR logs all events which take longer than 2 seconds at ERROR level. This will help identify events that are taking far too long, and which could be disruptive to the operation of the controller. The slow event logging looks like this:
```
[2024-12-20 15:03:39,754] ERROR [QuorumController id=1] Exceptionally slow controller event createTopics took 5240 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
Also, every 60 seconds, it logs some event time statistics, including average time, maximum time, and the name of the event which took the longest. This periodic message looks like this:
```
[2024-12-20 15:35:04,798] INFO [QuorumController id=1] In the last 60000 ms period, 333 events were completed, which took an average of 12.34 ms each. The slowest event was handleCommit[baseOffset=0], which took 41.90 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
An operator can disable these logs by adding the following to their log4j config:
```
org.apache.kafka.controller.EventPerformanceMonitor=OFF
```
Reviewers: Colin P. McCabe <cmccabe@apache.org>
When inter.broker.listener is explicitly set, validate that it is not in the set of controller.listener.names.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
It appears this test was failing because the transaction was never aborting and the concurrent transactions errors would not go away.
ccab9eb introduced the test failure because it requires the transaction to complete, but I suspect the lack of completion was happening before the change.
The timeout for the write is based on the transactional timeout, and 100ms seemed too small -- thus the requests to update the state would often repeatedly time out.
Also removed the loop since it was not necessary.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Calvin Liu <caliu@confluent.io>