This patch fixes the bug that allows the last known leader to be elected as a partition leader while still in a fenced state, before the next heartbeat removes the fence.
https://issues.apache.org/jira/browse/KAFKA-19522
Reviewers: Jun Rao <junrao@gmail.com>, TengYao Chi
<frankvicky@apache.org>
The MetadataImage has a lot of stuff in it and it gets passed around in
many places in the new GroupCoordinator. This makes it difficult to
understand what metadata the group coordinator actually relies on and
makes it too easy to use metadata in ways it wasn't meant to be used.
This change encapsulate the MetadataImage in an interface
(`CoordinatorMetadataImage`) that indicates and controls what metadata
the group coordinator actually uses. Now it is much easier at a glance
to see what dependencies the GroupCoordinator has on the metadata. Also,
now we have a level of indirection that allows more flexibility in how
the GroupCoordinator is provided the metadata it needs.
Removing the isEligibleLeaderReplicasV1Enabled to let ELR be enabled if
MV is at least 4.1IV1. Also bump the Latest Prod MV to 4.1IV1
Reviewers: Paolo Patierno <ppatierno@live.com>, Jun Rao <junrao@gmail.com>
- Updated `ClientQuotaImage` and `TopicImage` by using
`Collections.unmodifiableMap` or `ImmutableMap` to prevent accidental or
intentional mutations after construction.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
This PR adds the following metrics for each of the supported production
features (`metadata.version`, `kraft.version`, `transaction.version`,
etc.):
`kafka.server:type=MetadataLoader,name=FinalizedLevel,featureName=X`
`kafka.server:type=node-metrics,name=maximum-supported-level,feature-name=X`
`kafka.server:type=node-metrics,name=minimum-supported-level,feature-name=X`
Reviewers: Josep Prat <josep.prat@aiven.io>, PoAn Yang
<payang@apache.org>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Lan Ding
<isDing_L@163.com>, Chia-Ping Tsai <chia7712@gmail.com>
The PR do following:
1. Remove ReplicaManager#becomeLeaderOrFollower.
2. Remove `LeaderAndIsrRequest` and `LeaderAndIsrResponse`
3. Migrate `LeaderAndIsrRequest.PartitionState` to server-common module
and change to `PartitionState`
4. Remove `ControllerEpoch` from PartitionState
5. Remove `isShuttingDown` from BrokerServer and ReplicaManager
Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
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>
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.
This change compares the remote replica's HWM with the leader's HWM and
completes the FETCH request if the remote HWM is less than the leader's
HWM. When the leader's HWM is updated any pending FETCH RPC is
completed.
Reviewers: Alyssa Huang <ahuang@confluent.io>, David Arthur
<mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>
`CreateTopicPolicy#validate` may throw unexpected exception other than
`PolicyViolationException`. We should handle this case as well.
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Simplify Set initialization and reduce the overhead of creating extra
collections.
The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())
This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
This PR simplifies two ConcurrentHashMap fields by removing their Atomic
wrappers:
- Change `brokerContactTimesMs` from `ConcurrentHashMap<Integer,
AtomicLong>` to `ConcurrentHashMap<Integer, Long>`.
- Change `brokerRegistrationStates` from `ConcurrentHashMap<Integer,
AtomicInteger>` to `ConcurrentHashMap<Integer, Integer>`.
This removes mutable holders without affecting thread safety (see
discussion in #19828).
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
<frankvicky@apache.org>, Kevin Wu <kevin.wu2412@gmail.com>, Ken Huang
<s7133700@gmail.com>
In #19840, we broke de-duplication during ACL creation. This patch fixes
that and adds a test to cover this case.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Following the removal of the ZK-to-KRaft migration code in commit
85bfdf4, controller-to-broker communication is now handled by the
control-plane listener (`controller.listener.names`). The
`interBrokerListenerName` parameter in `ClusterControlManager` is no
longer referenced on the controller side and can be safely removed as
dead code.
Reviewers: Lan Ding <isDing_L@163.com>, Ken Huang <s7133700@gmail.com>,
Chia-Ping Tsai <chia7712@gmail.com>
Fix `updateBrokerContactTime` so that existing brokers still have their
contact time updated when they are already tracked. Also, update the
unit test to test this case.
Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Yung
<yungyung7654321@gmail.com>, TengYao Chi <frankvicky@apache.org>, Ken
Huang <s7133700@gmail.com>
This patch fixes a problem in AclControlManager where we are updating
the timeline data structures prematurely.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>, Andrew Schofield <aschofield@confluent.io>,
The main issue was that we forgot to set
`TopicConfig.SEGMENT_BYTES_CONFIG` to at least `1024 * 1024`, which
caused problems in tests with small segment sizes.
To address this, we introduced a new internal config:
`LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG`, allowing us to set smaller
segment bytes specifically for testing purposes.
We also updated the logic so that if a user configures the topic-level
segment bytes without explicitly setting the internal config, the
internal value will no longer be returned to the user.
In addition, we removed
`MetadataLogConfig#METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG` and added
three new internal configurations:
- `INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_DELETE_DELAY_MILLIS_CONFIG`
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
[KAFKA-16720](https://issues.apache.org/jira/browse/KAFKA-16720) aims to
finish the AlterShareGroupOffsets RPC.
Reviewers: Andrew Schofield <aschofield@confluent.io>
---------
Co-authored-by: jimmy <wangzhiwang@qq.com>
- Remove redundant close of `snapshotWriter`.
- `snapshotWriter` is already closed by `RaftSnapshotWriter#writer`.
Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This PR introduces the following per-broker metrics:
-`kafka.controller:type=KafkaController,name=BrokerRegistrationState,broker=X`
-`kafka.controller:type=KafkaController,name=TimeSinceLastHeartbeatReceivedMs,broker=X`
and this metric:
`kafka.controller:type=KafkaController,name=ControlledShutdownBrokerCount`
Reviewers: Colin P. McCabe <cmccabe@apache.org>
* In ConsoleShareConsumerTest, add `@SuppressWarnings("unchecked")`
annotation in method shouldUpgradeDeliveryCount
* In ListConsumerGroupOffsetsHandlerTest, add generic parameters to
HashSet constructors
* In TopicsImageTest, add explicit generic type to Collections.EMPTY_MAP
to fix raw type usage
Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
jira: https://issues.apache.org/jira/browse/KAFKA-19068
`RecordsIterator#decodeControlRecord` do the type check and then
`ControlRecord` constructor does that again.
we should add a static method to ControlRecord to create `ControlRecord`
with type check, and then `ControlRecord` constructor should be changed
to private to ensure all instance is created by the static method.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
The current ElectionWasClean checks if the new leader is in the previous
ISR. However, there is a corner case in the partition reassignment. The
partition reassignment can change the partition replicas. If the new
preferred leader (the first one in the new replicas) is the last one to
join ISR, this preferred leader will be elected in the same partition
change.
For example: In the previous state, the partition is Leader: 0,
Replicas (2,1,0), ISR (1,0), Adding(2), removing(0). Then replica 2
joins the ISR. The new partition would be like: Leader: 2, Replicas
(2,1), ISR(1,2). The new leader 2 is not in the previous ISR (1,0) but
it is still a clean election.
Reviewers: Jun Rao <junrao@gmail.com>
This PR adds an additional test case to `FormatterTest` that checks that
formatting with `--standalone` and then formatting again with
`--standalone --ignore-formatted` is indeed a no-op.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Log a message when reading a batch that is larger than the currently
allocated batch.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, PoAn Yang
<payang@apache.org>
replace all applicable `.stream().forEach()` in codebase with just
`.forEach()`.
Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
1. remove org.apache.kafka.raft.OffsetAndEpoch
2. rewrite org.apache.kafka.server.common.OffsetAndEpoch by record
keyword
3. rename OffsetAndEpoch#leaderEpoch to OffsetAndEpoch#epoch
Reviewers: PoAn Yang <payang@apache.org>, Xuan-Zhang Gong
<gongxuanzhangmelt@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The ShareVersion feature does not make any metadata version changes. As
a result, `SV_1` does not depend on any MV level, and no MV needs to be
defined for the preview of KIP-932.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Add new StreamsGroupFeature, disabled by default, and add "streams" as
default value to `group.coordinator.rebalance.protocols`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot
<david.jacot@gmail.com>, Lucas Brutschy <lbrutschy@confluent.io>,
Justine Olshan <jolshan@confluent.io>, Andrew Schofield
<aschofield@confluent.io>, Jun Rao <jun@confluent.io>
There will be an update to the PluginMetrics#metricName method: the type
of the tags parameter will be changed
from Map to LinkedHashMap.
This change is necessary because the order of metric tags is important
1. If the tag order is inconsistent, identical metrics may be treated as
distinct ones by the metrics backend
2. KAFKA-18390 is updating metric naming to use LinkedHashMap. For
consistency, we should follow the same approach here.
Reviewers: TengYao Chi <frankvicky@apache.org>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, lllilllilllilili
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>
This change implements upgrading the kraft version from 0 to 1 in existing clusters.
Previously, clusters were formatted with either version 0 or version 1, and could not
be moved between them.
The kraft version for the cluster metadata partition is recorded using the
KRaftVersion control record. If there is no KRaftVersion control record
the default kraft version is 0.
The kraft version is upgraded using the UpdateFeatures RPC. These RPCs
are handled by the QuorumController and FeatureControlManager. This
change adds special handling in the FeatureControlManager so that
upgrades to the kraft.version are directed to
RaftClient#upgradeKRaftVersion.
To allow the FeatureControlManager to call
RaftClient#upgradeKRaftVersion is a non-blocking fashion, the kraft
version upgrade uses optimistic locking. The call to
RaftClient#upgradeKRaftVersion does validations of the version change.
If the validations succeeds, it generates the necessary control records
and adds them to the BatchAccumulator.
Before the kraft version can be upgraded to version 1, all of the
brokers and controllers in the cluster need to support kraft version 1.
The check that all brokers support kraft version 1 is done by the
FeatureControlManager. The check that all of the controllers support
kraft version is done by KafkaRaftClient and LeaderState.
When the kraft version is 0, the kraft leader starts by assuming that
all voters do not support kraft version 1. The leader discovers which
voters support kraft version 1 through the UpdateRaftVoter RPC. The
KRaft leader handles UpdateRaftVoter RPCs by storing the updated
information in-memory until the kraft version is upgraded to version 1.
This state is stored in LeaderState and contains the latest directory
id, endpoints and supported kraft version for each voter.
Only when the KRaft leader has received an UpdateRaftVoter RPC from all
of the voters will it allow the upgrade from kraft.version 0 to 1.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
When the controller starts up (or becomes active after being inactive), we add all of the registered brokers to BrokerRegistrationTracker so that they will not be accidentally fenced the next time we are looking for a broker to fence. We do this because the state in BrokerRegistrationTracker is "soft state" (it doesn't appear in the metadata log), and the newly active controller starts off with no soft state. (Its soft state will be populated by the brokers sending heartbeat requests to it over time.)
In the case of fenced brokers, we are not worried about accidentally fencing the broker due to it being missing from
BrokerRegistrationTracker for a while (it's already fenced). Therefore, it should be reasonable to just not add fenced brokers to the tracker initially.
One case where this change will have a positive impact is for people running single-node demonstration clusters in combined KRaft mode. In that case, when the single-node cluster is taken down and restarted, it currently will have to wait about 9 seconds for the broker to come up and re-register. With this change, the broker should be able to re-register immediately (assuming the previous shutdown happened cleanly through controller shutdown.)
One possible negative impact is that if there is a controller failover, it will open a small window where a broker with the same ID as a fenced broker could re-register. However, our detection of duplicate broker IDs is best-effort (and duplicate broker IDs are an administrative mistake), so this downside seems acceptable.
Reviewers: Alyssa Huang <ahuang@confluent.io>, José Armando García Sancio <jsancio@apache.org>
This also adds metrics to StandardAuthorizer
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ken Huang
<s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TaiJuWu
<tjwu1217@gmail.com>
It has been around two weeks since fixing
QuorumControllerTest#testUncleanShutdownBrokerElrEnabled PR
https://github.com/apache/kafka/pull/19240 was merged. There is no flaky
result after 2025/03/21, so it has enough evidence to prove the flaky is
fixed. It's good to remove flaky tag.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
issue link https://issues.apache.org/jira/browse/KAFKA-15371
## conclusion
This issue isn’t caused by differences between the `log` file and the
`checkpoint` file, but rather by the order in which asynchronous events
occur.
## reliably reproduce
In the current version, you can reliably reproduce this issue by adding
a small sleep in `SnapshotFileReader#handleNextBatch` , like this:
```
private void handleNextBatch() {
if (!batchIterator.hasNext()) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
beginShutdown("done");
return;
}
FileChannelRecordBatch batch = batchIterator.next();
if (batch.isControlBatch()) {
handleControlBatch(batch);
} else {
handleMetadataBatch(batch);
}
scheduleHandleNextBatch();
lastOffset = batch.lastOffset();
}
```
you can download a test file [test checkpoint
file](https://github.com/user-attachments/files/19659636/00000000000000007169-0000000001.checkpoint.log)
⚠️: Please remove the .log extension after downloading, since GitHub
doesn’t allow uploading checkpoint files directly.
After change code and gradle build , you can run
`bin/kafka-metadata-shell.sh --snapshot ${your file path}`
You will only see a loading message in the console like this: <img
width="248" alt="image"
src="https://github.com/user-attachments/assets/fe4b4eba-7a6a-4cee-9b56-c82a5fa02c89"
/>
## Cause of the Bug
After the `SnapshotFileReader startup`, it will enqueue the iterator’s
events to its own kafkaQueue.
The impontent method is: `SnapshotFileReader#scheduleHandleNextBatch`
When processing each batch of the iterator, it adds metadata events for
the batch to the kafkaQueue(different from the SnapshotFileReader.) of
the metadataLoader. The impontent method is
`SnapshotFileReader#handleMetadataBatch` and
`MetadataLoader#handleCommit`
When the MetadataLoader processes a MetadataDelta, it checks whether the
high watermark has been updated. If not, it skips processing The
impontent method is `MetadataLoader#maybePublishMetadata` and
`maybePublishMetadata#stillNeedToCatchUp`
The crucial high watermark update happens after the SnapshotFileReader’s
iterator finishes reading, using the cleanup task of its kafkaQueue.
So, if the MetadataLoader finishes processing all batches before the
high watermark is updated, the main thread will keep waiting. <img
width="1088" alt="image"
src="https://github.com/user-attachments/assets/03daa288-ff39-49a3-bbc7-e7b5831a858b"
/>
<img width="867" alt="image"
src="https://github.com/user-attachments/assets/fc0770dd-de54-4f69-b669-ab4e696bd2a7"
/>
## Solution
If we’ve reached the last batch in the iteration, we update the high
watermark first before adding events to the MetadataLoader, ensuring
that MetadataLoader runs at least once after the watermark is updated.
After modifying the code, you’ll see the normal shell execution
behavior.
<img width="337" alt="image"
src="https://github.com/user-attachments/assets/2791d03c-81ae-4762-a015-4d6d9e526455"
/>
Reviewers: PoAn Yang <payang@apache.org>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This PR proposes a switch to enable share groups for 4.1 (preview) and
4.2 (GA).
* `share.version=1` to indicate that share groups are enabled. This is
used as the switch for turning share groups on and off.
In 4.1, the default will be `share.version=0`. Then a user wanting to
evaluate the preview of KIP-932 would use `bin/kafka-features.sh
--bootstrap.server xxxx upgrade --feature share.version=1`.
In 4.2, the default will be `share.version=1`.
Reviewers: Jun Rao <junrao@gmail.com>
Allow re-registration of brokers with active sessions if the previous broker registration was in controlled shutdown.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Reviewers: José Armando García Sancio <jsancio@apache.org>, David Mao <dmao@confluent.io>
Minor improvement on the error output for the storage format command. Suggests changes for a valid storage format command.
Reviewers: Justine Olshan <jolshan@confluent.io>
Add a metric to track the number of election is done using ELR.
https://issues.apache.org/jira/browse/KAFKA-18954
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Justine Olshan
<jolshan@confluent.io>
Given that now we support Java 17 on our brokers, this PR replace the
use of the following in metadata module:
Collections.singletonList() and Collections.emptyList() with List.of()
Collections.singletonMap() and Collections.emptyMap() with Map.of()
Collections.singleton() and Collections.emptySet() with Set.of()
Reviewers: David Arthur <mumrah@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Skip kraft.version when applying FeatureLevelRecord records. The kraft.version is stored as control records and not as metadata records. This solution has the benefits of removing from snapshots any FeatureLevelRecord for kraft.version that was incorrectly written to the log and allows ApiVersions to report the correct finalized kraft.version.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The `FeatureControlManager` used `MetadataVersion#LATEST_PRODUCTION` as uninitialized MV. This makes other component may get a stale MV. In production code, the `FeatureControlManager` set MV when replaying `FeatureLevelRecord`, so we can set `Optional.empty()` as uninitialized MV. If other components get an empty result, the `FeatureLevelRecord` throws an exception like `FeaturesImage`.
Unit test:
* FeatureControlManagerTest#testMetadataVersion: test getting MetadataVersion
* before and after replaying FeatureLevelRecord.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>