Commit Graph

440 Commits

Author SHA1 Message Date
Kevin Wu 099e91f5fc
KAFKA-19719 --no-initial-controllers should not assume kraft.version=… (#20616)
CI / build (push) Has been cancelled Details
backport KAFKA-19719 to 4.0

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-06 22:42:35 +08:00
Calvin Liu 46e843da9f
KAFKA-19383: Handle the deleted topics when applying ClearElrRecord (#20034)
CI / build (push) Waiting to run Details
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>

No conflicts.
2025-06-24 19:59:50 -07:00
Alyssa Huang 52a5b88512 KAFKA-19411: Fix deleteAcls bug which allows more deletions than max records per user op (#19974)
CI / build (push) Waiting to run Details
If there are more deletion filters after we initially hit the
`MAX_RECORDS_PER_USER_OP` bound, we will add an additional deletion
record ontop of that for each additional filter.

The current error message returned to the client is not useful either,
adding logic so client doesn't just get `UNKNOWN_SERVER_EXCEPTION` with
no details returned.
2025-06-24 16:00:20 -07:00
Colin Patrick McCabe 0297ba2c67 KAFKA-19192; Old bootstrap checkpoint files cause problems updated servers (#19545)
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>
2025-04-24 16:55:22 -04:00
José Armando García Sancio 4990c970c1 KAFKA-18979; Report correct kraft.version in ApiVersions (#19205)
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>
2025-03-13 18:44:52 -04:00
Calvin Liu d671e7dbc5 KAFKA-18940: fix electionWasClean (#19156)
The electionWasClean should also consider if the election is done
through ELR. Otherwise, the metric uncleanLeaderElection will wrongly
count the ELR election
https://issues.apache.org/jira/browse/KAFKA-18940

Reviewers: Jun Rao <junrao@gmail.com>
2025-03-07 13:47:50 -08:00
Colin Patrick McCabe 5c0b8bef07 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:47:05 -08:00
PoAn Yang 69186480dc KAFKA-18844: Stale features information in QuorumController#registerBroker (#18997)
In https://github.com/apache/kafka/pull/16848, we added `kraft.version`
to finalized features and got finalized features outside controller
event handling thread. This may make finalized features stale when
processing `registerBroker` event. Also, some cases like
`QuorumControllerTest.testBalancePartitionLeaders` become flaky cause of
outdated MV. This PR moves finalized features back to controller event
handling thread to avoid the error.

Reviewers: Ismael Juma <ijuma@apache.org>, Jun Rao <junrao@gmail.com>,
Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-03-03 15:17:01 +01:00
Xuan-Zhang Gong 9516b3bdd0 KAFKA-18864:remove the Evolving tag from stable public interfaces (#19036)
The purpose of this PR is to remove the `@InterfaceStability.Evolving` from classes that were created over a year ago.

Reviewers: Jun Rao <junrao@gmail.com>
2025-03-03 15:14:33 +01:00
Calvin Liu 8383c88a70 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:47 +01:00
S.Y. Wang ba52124a87 MINOR: Fix typo in ClusterControlManager (#18886)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 09:57:26 -08:00
Calvin Liu 909fae1aaa MINOR: Fix incorrect return value from upgradeFeatures #18958
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-02-19 09:44:05 -08:00
Ismael Juma 3fbbd0a3ee KAFKA-18601: Assume a baseline of 3.3 for server protocol versions (#18845)
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>
2025-02-19 05:57:04 -08:00
Jhen-Yung Hsu 8797c903ee KAFKA-18803 The acls would appear at the wrong level of the metadata shell "tree" (#18916)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 03:56:53 +08:00
David Jacot e37af20760 MINOR: Mark IBP_4_0_IV3 as production ready! (#18902)
This patch marks IBP_4_0_IV3 as production ready for the Apache Kafka 4.0 release. It also introduced IBP_4_1_IV0 as the next development version.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-02-14 17:17:30 +01:00
Calvin Liu ba067caa54 KAFKA-18634: Fix ELR metadata version issues (#18680)
This patch cleans up the places that should not use MV to determine ELR is enabled marks 4.0IV1 stable.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-02-14 08:41:05 +01:00
Ken Huang ad6db0952b KAFKA-18225 ClientQuotaCallback#updateClusterMetadata is unsupported by kraft (#18196)
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>
2025-02-11 01:07:28 +08:00
kevin-wu24 98d238aef0 KAFKA-16524; Metrics for KIP-853 (#18304)
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>
2025-02-04 14:19:11 -08:00
Calvin Liu 226532a966 KAFKA-18635: reenable the unclean shutdown detection (#18277)
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>
2025-02-04 14:07:27 -08:00
Calvin Liu eb352ff00b KAFKA-18649: complete ClearElrRecord handling (#18708)
Implement ClearElrRecord handling in the TopicDelta. Also, the ReplicationControlManager should not merge updates if ELR/LastKnownElr are empty, becuase that will cause an unnecessary partition epoch bump.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-02-04 13:56:01 -08:00
Calvin Liu ffc57421f5 KAFKA-16540: Clear ELRs when min.insync.replicas is changed. (#18148)
In order to maintain the integrity of replication, we need to clear the ELRs of affected partitions when min.insync.replicas is changed. This could happen at the topic level, or at a global level if the cluster level default is changed.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-02-04 13:51:32 -08:00
Calvin Liu 501043f737 KAFKA-16540: enforce min.insync.replicas config invariants for ELR (#17952)
If ELR is enabled, we need to set a cluster-level min.insync.replicas, and remove all broker-level overrides. The reason for this is that if brokers disagree about which partitions are under min ISR, it breaks the KIP-966 replication invariants. In order to enforce this, when the eligible.leader.replicas.version feature is turned on, we automatically remove all broker-level min.insync.replicas overrides, and create the required cluster-level override if needed. Similarly, if the cluster was created with eligible.leader.replicas.version enabled, we create a similar cluster-level record. In both cases, we don't allow setting overrides for individual brokers afterwards, or removing the cluster-level override.

Split ActivationRecordsGeneratorTest up into multiple test cases rather than having it be one giant test case.

Fix a bug in QuorumControllerTestEnv where we would replay records manually on objects, racing with the active controller thread. Instead, we should simply ensure that the initial bootstrap records contains what we want.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-02-04 13:02:41 -08:00
David Arthur 0d2fd09f83 KAFKA-16446: Improve controller event duration logging (#15622)
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>
2025-02-04 12:55:30 -08:00
Ken Huang e13268c77b KAFKA-18598: Remove ControllerMetadataMetrics ZK-related Metrics (#18629)
Reviewers: Christo Lolov <lolovc@amazon.com>
2025-01-22 14:04:51 +00:00
Ismael Juma 4ce4ac7301 KAFKA-14552: Assume a baseline of 3.0 for server protocol versions (#18497)
Kafka 4.0 will remove support for zk mode and will require conversion to kraft
before upgrading to 4.0. The minimum kraft version is 3.0 (aka 3.0-IV1).

This provides an opportunity to remove exclusively server side protocols versions
that only exist to allow direct upgrades from versions older than 3.0 or that are
used only by zk mode.

Since KRaft became production ready in 3.3, we should consider setting the
baseline to 3.3. But that requires more discussion and it can be done via a
separate change (KAFKA-18601).

Protocol changes:
* Remove RequestHeader v0 (only used by ControlledShutdown v0)
* Remove WriteTxnMarkers v0
* Remove all versions of ControlledShutdown, LeaderAndIsr, StopReplica, UpdateMetadata

In order to remove all versions safely, extend generator to support setting
"versions" to "none". In this case, we no longer generate the `*Data` classes,
but we still reserve the id for the relevant protocol api (so it doesn't get
accidentally used for something else). The protocol documentation is correct
after these changes.

We kept a simplified version of `LeaderAndIsr{Request|Response}` because
it's used by many tests that are still relevant in kraft mode. Once
KAFKA-18486 is done, it may be possible to remove it (I left a comment on
the ticket). Similarly, KAFKA-18487 may make it possible to remove
the introduced `StopReplicaPartitionState` (left a comment on that ticket too).

There are a number of places that were adjusted to include an
`ApiKeys.hasValidVersion` check.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-20 18:23:39 -08:00
Ismael Juma 04900e761b KAFKA-18465: Remove MetadataVersions older than 3.0-IV1 (#18468)
Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.

Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
2025-01-11 09:43:13 -08:00
Justine Olshan 2c4d2a3be5 KAFKA-17705: Add Transactions V2 system tests and mark as production ready (#18132)
Added transaction version 2 to some of the system tests. Also marking TV2 as production ready.

Also fixes the defaultVersion test. 

Reviewers: Jun Rao <jun@confluent.io>
2024-12-21 14:02:50 -08:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
Gantigmaa Selenge 747dc172e8
KIP-1073: Return fenced brokers in DescribeCluster response (#17524)
mplementation of KIP-1073: Return fenced brokers in DescribeCluster response.
Add new unit and integration tests for describeCluster.

Reviewers: Luke Chen <showuon@gmail.com>
2024-12-13 10:58:11 +08:00
Nick Guo 671cbedc1b
KAFKA-18219 Use INFO level instead of ERROR after successfully performing an unclean leader election (#18159)
Reviewers: Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-13 03:57:14 +08:00
TengYao Chi 772aa241b2
KAFKA-18136: Remove zk migration from code base (#18016)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-12 18:34:29 +01:00
David Mao 0ff55c316a
KAFKA-18106: Generate LeaderAndIsrUpdates on unclean shutdown (#18045)
Generate LeaderAndISR change records when a broker re-registers and the quorum controller detects an unclean shutdown.

This is necessary to ensure that we perform the expected partition state transitions, eg: bumping leader epochs and so on.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-12-05 16:19:05 -08:00
Dongnuo Lyu e30edb3eff
KAFKA-18052: Decouple the dependency of feature stable version to the metadata version (#17886)
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).

This patch introduces a LATEST_PRODUCTION value independent from the metadata version to each feature so that the highest supported feature version can be customized by the feature owner.

The change only applies to dynamic feature upgrade. During formatting, we still use the default value associated the metadata version.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2024-12-05 11:07:47 -08:00
Ken Huang 2b43c49f51
KAFKA-18050 Upgrade the checkstyle version to 10.20.2 (#17999)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-05 10:59:18 +08:00
Colin Patrick McCabe a8cdbaf4b3
KAFKA-18138: The controller must add all extant brokers to BrokerHeartbeatTracker when activating (#18009)
The controller must add all extant brokers to BrokerHeartbeatTracker when activating. Otherwise, we
could end up in a situation where a broker fails exactly as a controller failover occurs, and we
never fence it.

Also, fix a bug where the slf4j logger object in PeriodicTaskControlManager was initialized as
though it belonged to OffsetControlManager.

Reviewers: David Mao <dmao@confluent.io>, David Arthur <mumrah@gmail.com>
2024-12-03 10:33:52 -05:00
Calvin Liu 2b2b3cd355
KAFKA-18062: use feature version to enable ELR (#17867)
Replace the ELR static config with feature version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-26 14:40:23 -08:00
PoAn Yang 98d47f47ef
KAFKA-18028 the effective kraft version of --no-initial-controllers should be 1 rather than 0 (#17836)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-27 01:45:11 +08:00
Colin Patrick McCabe cd36d64535
KAFKA-18051: Disallow creating ACLs with principals that do not contain a colon (#17883)
Kafka Principals must contain a colon. We should enforce this in createAcls.

Reviewers: David Arthur <mumrah@gmail.com>
2024-11-22 16:50:33 -08:00
Colin Patrick McCabe 130bf1054b
MINOR: some minor cleanups in the quorum controller. (#17819)
BrokerHeartbeatManager.java: fix an outdated comment.

Move an inefficient test method that is O(num_brokers) from ClusterControlManager.java into ReplicationControlManagerTest.java, so that it doesn't accidentally get used in production code.

Remove QuorumController.ImbalanceSchedule, etc. since it is no longer used.

Move the initialization of OffsetControlManager later in the QuorumController constructor and add a comment explaining why it should come last. This doesn't fix any bugs currently, but it's a good practice for the future.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-11-18 11:15:38 -08:00
Colin Patrick McCabe 085b27ec6e
KAFKA-17987 Remove assorted ZK-related files (#17768)
Remove zookeeper files in bin:
- bin/zookeeper-security-migration.sh
- bin/zookeeper-server-start.sh
- bin/zookeeper-server-stop.sh
- bin/zookeeper-shell.sh

Remove files used to configure Kafka in zookeeper mode in config:
- config/server.properties
- config/zookeeper.properties

Remove ZK references from all remaining Kafka configuration files.

Remove ZK references from all log4j.properties files.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-13 20:32:18 +08:00
kevin-wu24 ebb3202e01
KAFKA-16964 Integration tests for adding and removing voters (#17582)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-05 03:09:37 +08:00
Mahsa Seifikar b864a66439
MINOR: Add logging for ReplicationControlManager topic deletion (#17617)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-01 12:24:22 -07:00
Jonah Hooper 18b8b992f9
[KAFKA-17870] Fail CreateTopicsRequest if total number of partitions exceeds 10k (#17604)
We fail the entire CreateTopicsRequest action if there are more than 10k total
partitions being created in this topic for this specific request. The usual pattern for
this API to try and succeed with some topics. Since the 10k limit applies to all topics
then no topic should be created if they all exceede it.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-10-31 13:54:03 -07:00
Mickael Maison d7135b2a5b
MINOR: Various cleanups in metadata (#17633)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-31 02:48:33 +08:00
Colin Patrick McCabe 14a9130f6f
KAFKA-17793: Improve kcontroller robustness against long delays (#17502)
As described in KIP-500, the Kafka controller monitors the liveness of each broker in the cluster. It gathers this information from heartbeats sent from the brokers themselves.

In some rare cases, the main controller thread may get blocked for several seconds at a time. In the current code, this will result in the controller being unable to update the last contact times for the brokers during this time.

This PR changes the controller heartbeat handling to be partially lockless. Specifically, the last contact time for each broker will be updated locklessly prior to the rest of the heartbeat handling. This will ensure that heartbeats always get through.

Additionally, this PR adds a PeriodicTaskControlManager to better manage periodic tasks. This should help handle the very common pattern where we want to schedule a background task at some frequency. We also want the background task to be immediately rescheduled if there is too much work to be done in one event.

Reviewers: Liu Zeyu <zeyu.luke@gmail.com>, David Arthur <mumrah@gmail.com>
2024-10-28 08:36:07 -07:00
Kuan-Po Tseng edb623cf67
MINOR: Remove unused method in BrokerRegistration (#17568)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-23 02:12:20 +08:00
Colin Patrick McCabe e3751a838c
KAFKA-17794: Add some formatting safeguards for KIP-853 (#17504)
KIP-853 adds support for dynamic KRaft quorums. This means that the quorum topology is
no longer statically determined by the controller.quorum.voters configuration. Instead, it
is contained in the storage directories of each controller and broker.

Users of dynamic quorums must format at least one controller storage directory with either
the --initial-controllers or --standalone flags.  If they fail to do this, no quorum can be
established. This PR changes the storage tool to warn about the case where a KIP-853 flag has
not been supplied to format a KIP-853 controller. (Note that broker storage directories
can continue to be formatted without a KIP-853 flag.)

There are cases where we don't want to specify initial voters when formatting a controller. One
example is where we format a single controller with --standalone, and then dynamically add 4
more controllers with no initial topology. In this case, we want the 4 later controllers to grab
the quorum topology from the initial one. To support this case, this PR adds the
--no-initial-controllers flag.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Federico Valeri <fvaleri@redhat.com>
2024-10-21 10:06:41 -07:00
Eric Chang 6b28e81ba1
KAKFA-17173 move quota config params from KafkaConfig to QuotaConfig (#17505)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-19 18:01:06 +08:00
Gaurav Narula b03fe66cfe
KAFKA-17759 Remove Utils.mkSet (#17460)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 21:20:43 +08:00
Chia-Chuan Yu b2380d7bf6
KAFKA-17772 Remove inControlledShutdownBrokers(Set<Integer>) and unfenceBrokers(Set<Integer>) from ReplicationControlManagerTest (#17466)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 17:14:12 +08:00