(kafka) branch trunk updated: KAFKA-16520: Support KIP-853 in DescribeQuorum (#16106)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new aecaf444756 KAFKA-16520: Support KIP-853 in DescribeQuorum (#16106) aecaf444756 is described below commit aecaf4447561edd8da9f06e3abdf46f382dc9d89 Author: Nikolay AuthorDate: Tue Jun 11 20:01:35 2024 +0300 KAFKA-16520: Support KIP-853 in DescribeQuorum (#16106) Add support for KIP-953 KRaft Quorum reconfiguration in the DescribeQuorum request and response. Also add support to AdminClient.describeQuorum, so that users will be able to find the current set of quorum nodes, as well as their directories, via these RPCs. Reviewers: Luke Chen , Colin P. McCabe , Andrew Schofield --- .../kafka/clients/admin/KafkaAdminClient.java | 20 +- .../org/apache/kafka/clients/admin/QuorumInfo.java | 76 ++- .../common/requests/DescribeQuorumRequest.java | 4 +- .../common/requests/DescribeQuorumResponse.java| 13 +- .../common/message/DescribeQuorumRequest.json | 4 +- .../common/message/DescribeQuorumResponse.json | 24 +- .../kafka/clients/admin/KafkaAdminClientTest.java | 23 +- .../unit/kafka/server/ApiVersionsRequestTest.scala | 4 +- .../kafka/server/DescribeQuorumRequestTest.scala | 2 + .../scala/unit/kafka/server/KafkaApisTest.scala| 3 +- .../org/apache/kafka/raft/KafkaRaftClient.java | 10 +- .../apache/kafka/raft/KafkaRaftClientDriver.java | 1 + .../java/org/apache/kafka/raft/LeaderState.java| 84 +-- .../java/org/apache/kafka/raft/QuorumState.java| 3 +- .../java/org/apache/kafka/raft/RaftMessage.java| 3 +- .../java/org/apache/kafka/raft/RaftRequest.java| 17 +- .../java/org/apache/kafka/raft/RaftResponse.java | 5 + .../kafka/raft/internals/BlockingMessageQueue.java | 5 + .../org/apache/kafka/raft/internals/VoterSet.java | 6 +- .../apache/kafka/raft/KafkaNetworkChannelTest.java | 1 + .../org/apache/kafka/raft/KafkaRaftClientTest.java | 7 + .../org/apache/kafka/raft/LeaderStateTest.java | 243 - .../apache/kafka/raft/RaftClientTestContext.java | 19 +- .../apache/kafka/raft/RaftEventSimulationTest.java | 2 +- .../kafka/raft/internals/KafkaRaftMetricsTest.java | 4 +- 25 files changed, 418 insertions(+), 165 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index c59cccf67c4..d7d525e4431 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -4414,12 +4414,13 @@ public class KafkaAdminClient extends AdminClient { private QuorumInfo.ReplicaState translateReplicaState(DescribeQuorumResponseData.ReplicaState replica) { return new QuorumInfo.ReplicaState( replica.replicaId(), +replica.replicaDirectoryId() == null ? Uuid.ZERO_UUID : replica.replicaDirectoryId(), replica.logEndOffset(), replica.lastFetchTimestamp() == -1 ? OptionalLong.empty() : OptionalLong.of(replica.lastFetchTimestamp()), replica.lastCaughtUpTimestamp() == -1 ? OptionalLong.empty() : OptionalLong.of(replica.lastCaughtUpTimestamp())); } -private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) { +private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition, DescribeQuorumResponseData.NodeCollection nodeCollection) { List voters = partition.currentVoters().stream() .map(this::translateReplicaState) .collect(Collectors.toList()); @@ -4428,12 +4429,21 @@ public class KafkaAdminClient extends AdminClient { .map(this::translateReplicaState) .collect(Collectors.toList()); +Map nodes = nodeCollection.stream().map(n -> { +List endpoints = n.listeners().stream() +.map(l -> new RaftVoterEndpoint(l.name(), l.host(), l.port())) +.collect(Collectors.toList()); + +return new QuorumInfo.Node(n.nodeId(), endpoints); +}).collect(Collectors.toMap(QuorumInfo.Node::nodeId, Function.identity())); + return new QuorumInfo( partition.leaderId(), partition.leaderEpoch(), partition.highWatermark(), voters, -observers +observers, +
(kafka) branch 3.8 updated: KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.8 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.8 by this push: new 7879f1c0131 KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs 7879f1c0131 is described below commit 7879f1c01311912be5a055045111f1a628c47c60 Author: Colin P. McCabe AuthorDate: Tue Jun 4 14:04:59 2024 -0700 KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs Implement the add voter, remove voter, and update voter RPCs for KIP-853. This is just adding the RPC handling; the current implementation in RaftManager just throws UnsupportedVersionException. Reviewers: Andrew Schofield , José Armando García Sancio Conflicts: Fix some conflicts caused by the lack of KIP-932 RPCs in 3.8. --- .../kafka/clients/admin/AddRaftVoterOptions.java | 26 ++ .../kafka/clients/admin/AddRaftVoterResult.java| 42 + .../java/org/apache/kafka/clients/admin/Admin.java | 56 .../kafka/clients/admin/ForwardingAdmin.java | 10 +++ .../kafka/clients/admin/KafkaAdminClient.java | 100 + .../kafka/clients/admin/RaftVoterEndpoint.java | 100 + .../clients/admin/RemoveRaftVoterOptions.java | 26 ++ .../kafka/clients/admin/RemoveRaftVoterResult.java | 42 + .../org/apache/kafka/common/protocol/ApiKeys.java | 6 +- .../kafka/common/requests/AbstractRequest.java | 6 ++ .../kafka/common/requests/AbstractResponse.java| 6 ++ .../kafka/common/requests/AddRaftVoterRequest.java | 75 .../common/requests/AddRaftVoterResponse.java | 65 ++ .../common/requests/RemoveRaftVoterRequest.java| 75 .../common/requests/RemoveRaftVoterResponse.java | 65 ++ .../common/requests/UpdateRaftVoterRequest.java| 73 +++ .../common/requests/UpdateRaftVoterResponse.java | 65 ++ .../common/message/AddRaftVoterRequest.json| 40 + .../common/message/AddRaftVoterResponse.json | 30 +++ .../common/message/RemoveRaftVoterRequest.json | 30 +++ .../common/message/RemoveRaftVoterResponse.json| 30 +++ .../common/message/UpdateRaftVoterRequest.json | 46 ++ .../common/message/UpdateRaftVoterResponse.json| 28 ++ .../kafka/clients/admin/MockAdminClient.java | 10 +++ .../kafka/common/requests/RequestResponseTest.java | 64 + .../scala/kafka/network/RequestConvertToJson.scala | 6 ++ core/src/main/scala/kafka/raft/RaftManager.scala | 5 +- .../main/scala/kafka/server/ControllerApis.scala | 18 core/src/main/scala/kafka/server/KafkaApis.scala | 2 + .../scala/unit/kafka/server/RequestQuotaTest.scala | 9 ++ .../scala/unit/kafka/tools/StorageToolTest.scala | 10 ++- 31 files changed, 1161 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java new file mode 100644 index 000..d917c09fa1e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * Options for {@link Admin#addRaftVoter}. + */ +@InterfaceStability.Stable +public class AddRaftVoterOptions extends AbstractOptions { +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterResult.java new file mode 100644 index 000..d42204c5e4e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterResult.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional i
(kafka) branch trunk updated (8b3c77c6711 -> 9ceed8f18f4)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git from 8b3c77c6711 KAFKA-15305 The background thread should try to process the remaining task until the shutdown timer is expired. (#16156) add 9ceed8f18f4 KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs No new revisions were added by this update. Summary of changes: .../admin/AddRaftVoterOptions.java}| 12 +-- ...ionTokenResult.java => AddRaftVoterResult.java} | 20 ++--- .../java/org/apache/kafka/clients/admin/Admin.java | 56 .../kafka/clients/admin/ForwardingAdmin.java | 10 +++ .../kafka/clients/admin/KafkaAdminClient.java | 100 + .../kafka/clients/admin/RaftVoterEndpoint.java | 100 + .../admin/RemoveRaftVoterOptions.java} | 12 +-- ...TokenResult.java => RemoveRaftVoterResult.java} | 20 ++--- .../org/apache/kafka/common/protocol/ApiKeys.java | 5 +- .../kafka/common/requests/AbstractRequest.java | 6 ++ .../kafka/common/requests/AbstractResponse.java| 6 ++ ...nTokenRequest.java => AddRaftVoterRequest.java} | 65 +++--- ...DirsResponse.java => AddRaftVoterResponse.java} | 37 ...irsRequest.java => RemoveRaftVoterRequest.java} | 48 +- ...nResponse.java => RemoveRaftVoterResponse.java} | 26 +++--- ...irsRequest.java => UpdateRaftVoterRequest.java} | 46 +- ...nResponse.java => UpdateRaftVoterResponse.java} | 26 +++--- .../common/message/AddRaftVoterRequest.json| 40 + ...rIdsResponse.json => AddRaftVoterResponse.json} | 12 ++- ...onsRequest.json => RemoveRaftVoterRequest.json} | 17 ++-- ...sResponse.json => RemoveRaftVoterResponse.json} | 12 ++- ...ionRequest.json => UpdateRaftVoterRequest.json} | 37 ...yResponse.json => UpdateRaftVoterResponse.json} | 18 ++-- .../kafka/clients/admin/MockAdminClient.java | 10 +++ .../kafka/common/requests/RequestResponseTest.java | 64 + .../scala/kafka/network/RequestConvertToJson.scala | 6 ++ core/src/main/scala/kafka/raft/RaftManager.scala | 5 +- .../main/scala/kafka/server/ControllerApis.scala | 18 core/src/main/scala/kafka/server/KafkaApis.scala | 2 + .../scala/unit/kafka/server/RequestQuotaTest.scala | 9 ++ .../scala/unit/kafka/tools/StorageToolTest.scala | 10 ++- 31 files changed, 648 insertions(+), 207 deletions(-) copy clients/src/main/java/org/apache/kafka/{server/authorizer/AuthorizationResult.java => clients/admin/AddRaftVoterOptions.java} (81%) copy clients/src/main/java/org/apache/kafka/clients/admin/{ExpireDelegationTokenResult.java => AddRaftVoterResult.java} (68%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/RaftVoterEndpoint.java copy clients/src/main/java/org/apache/kafka/{server/authorizer/AuthorizationResult.java => clients/admin/RemoveRaftVoterOptions.java} (80%) copy clients/src/main/java/org/apache/kafka/clients/admin/{ExpireDelegationTokenResult.java => RemoveRaftVoterResult.java} (68%) copy clients/src/main/java/org/apache/kafka/common/requests/{CreateDelegationTokenRequest.java => AddRaftVoterRequest.java} (55%) copy clients/src/main/java/org/apache/kafka/common/requests/{AssignReplicasToDirsResponse.java => AddRaftVoterResponse.java} (64%) copy clients/src/main/java/org/apache/kafka/common/requests/{DescribeLogDirsRequest.java => RemoveRaftVoterRequest.java} (55%) copy clients/src/main/java/org/apache/kafka/common/requests/{ControllerRegistrationResponse.java => RemoveRaftVoterResponse.java} (64%) copy clients/src/main/java/org/apache/kafka/common/requests/{DescribeLogDirsRequest.java => UpdateRaftVoterRequest.java} (56%) copy clients/src/main/java/org/apache/kafka/common/requests/{ControllerRegistrationResponse.java => UpdateRaftVoterResponse.java} (64%) create mode 100644 clients/src/main/resources/common/message/AddRaftVoterRequest.json copy clients/src/main/resources/common/message/{AllocateProducerIdsResponse.json => AddRaftVoterResponse.json} (74%) copy clients/src/main/resources/common/message/{GetTelemetrySubscriptionsRequest.json => RemoveRaftVoterRequest.json} (65%) copy clients/src/main/resources/common/message/{AllocateProducerIdsResponse.json => RemoveRaftVoterResponse.json} (74%) copy clients/src/main/resources/common/message/{ControllerRegistrationRequest.json => UpdateRaftVoterRequest.json} (51%) copy clients/src/main/resources/common/message/{PushTelemetryResponse.json => UpdateRaftVoterResponse.json} (77%)
(kafka) branch KAFKA-16649 deleted (was 32eb8c3d68d)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16649 in repository https://gitbox.apache.org/repos/asf/kafka.git was 32eb8c3d68d KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable The revisions that were on this branch are still contained in other references; therefore, this change does not discard any commits from the repository.
(kafka) branch KAFKA-16649 updated (f435287ac63 -> 32eb8c3d68d)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16649 in repository https://gitbox.apache.org/repos/asf/kafka.git discard f435287ac63 Fixes omit 5a7a9b7f213 KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable add 31355ef8f94 KAFKA-16475: add more tests to TopicImageNodeTest (#15735) add 4825c89d14e KAFKA-16588 broker shutdown hangs when log.segment.delete.delay.ms is zero (#15773) add d9c36299db7 KAFKA-16614 Disallow @ClusterTemplate("") (#15800) add 89d8045a15b KAFKA-16647 Remove setMetadataDirectory from BrokerNode/ControllerNode (#15833) add cdc4caa5787 KAFKA-14588 UserScramCredentialsCommandTest rewritten in Java (#15832) add a3f24149905 KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810) add 87390f961f3 KAFKA-16572 allow defining number of disks per broker in ClusterTest (#15745) add 240243b91d6 KAFKA-10199: Accept only one task per element in output queue for failed tasks (#15849) add 2c0b8b69207 MINOR: ConsumerGroup#getOrMaybeCreateMember should not add the member to the group (#15847) add 1fd39150aa3 KAFKA-16655: Deflake ZKMigrationIntegrationTest.testDualWrite #15845 add 9b8aac22ec7 KAFKA-16427 KafkaConsumer#position() does not respect timeout when group protocol is CONSUMER (#15843) add bfe81d62297 KAFKA-16207; KRaft's internal log listener to update voter set (#15671) add 25118cec145 MINOR: remove redundant check in KafkaClusterTestKit (#15858) add 970ac078812 KAFKA-16659 KafkaConsumer#position() does not respect wakup when group protocol is CONSUMER (#15853) add 41f5bf844df KAFKA-16223 Replace EasyMock/PowerMock with Mockito for KafkaConfigBackingStoreTest (2/3) (#15841) add 55a00be4e97 MINOR: Replaced Utils.join() with JDK API. (#15823) add 366aeab488c KAFKA-10199: Add remove operation with future to state updater (#15852) add 6a8977e2125 KAFKA-14588 [3/N] ConfigCommandTest rewritten in java (#15850) add 5c96ad61d95 KAFKA-16393 read/write sequence of buffers correctly (#15571) add 42754336e1f MINOR: Remove `ConsumerGroupPartitionMetadataValue.Epoch` field (#15854) add 0b4eaefd863 KAFKA-16646: KAFKA-16646Don't run cve scan job on forks (#15831) add aeca384641b KAFKA-16356: Remove class-name dispatch in RemoteLogMetadataSerde (#15620) add 4c4ae6e39c4 KAFKA-16608 Honour interrupted thread state on KafkaConsumer.poll (#15803) add fe8ccbc92c2 KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration (#15744) add 0df340d64d3 KAFKA-16470 kafka-dump-log --offsets-decoder should support new records (#15652) add 0de3b7c40b0 KAFKA-16593 Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions (#15766) add d76352e2151 MINOR: log newly created processId (#15851) add 459eaec666c KAFKA-16615; JoinGroup API for upgrading ConsumerGroup (#15798) add ea485a70611 KAFKA-16665: Allow to initialize newly assigned partition's positions without allowing fetching while callback runs (#15856) add cb35ddc5ca2 KAFKA-10199: Remove lost tasks in state updater with new remove (#15870) add 21bf715622e KAFKA-16307; Fix coordinator thread idle ratio (#15835) add 05df10449eb KAFKA-13328, KAFKA-13329 (2): Add custom preflight validation support for connector header, key, and value converters (#14309) add a4c6cefd10f KAFKA-14226: Introduce FieldPath abstraction and nested path support for ExtractField SMT (#15379) add 525b9b1d768 KAFKA-15018: Write connector tombstone offsets to secondary store before primary store (#13801) add a0f1658bb13 KAFKA-16678 Remove variable "unimplementedquorum" (#15879) add 5f933ac8403 MINOR: Correct connector scheduled rebalance logs (#15875) add 8655094e6c9 KAFKA-16511: Fix the leaking tiered segments during segment deletion (#15817) add 3b43edd7a1c MINOR: Remove dev_version parameter from streams tests (#15874) add f7b242f94e8 KAFKA-10199: Revoke tasks from state updater with new remove (#15871) add f74f596bc7d KAFKA-16640 Replace TestUtils#resource by scala.util.Using (#15881) add 2a5efe4a334 KAFKA-16685: Add parent exception to RLMTask warning logs (#15880) add c64a315fd55 MINOR: Made the supportedOperation variable name more verbose (#15892) add 29f3260a9c0 MINOR: Fix streams javadoc links (#15900) add 397d58b14c9 MINOR: use classic consumer with ZK mode for DeleteOffsetsConsumerGroupCommandIntegrationTest (#15872) add f4fdaa702a2 MINOR: Add missing RPCs to security.html (#15878) add 5a9ccb6b773 KAFKA-16445: Add PATCH method for connector config (#6934) add 8fd6596454e KAFKA-15045: (KIP-924) New interfaces and stubbed utility classes for pluggable TaskAssignors. (#15887) add 7e9ab4b2c68 KAFKA-16484 Support to define per broker/controller property by ClusterConfigProperty (#15715)
(kafka) 01/01: KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16649 in repository https://gitbox.apache.org/repos/asf/kafka.git commit 32eb8c3d68d4d6fc30705a55371cfe2ee57b8f77 Author: Colin P. McCabe AuthorDate: Tue Apr 30 14:05:32 2024 -0700 KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable Do not acquire the DynamicBrokerConfig lock in DynamicBrokerConfig.removeReconfigurable. It's not necessary, because the list that these functions are modifying is a thread-safe CopyOnWriteArrayList. In DynamicBrokerConfig.reloadUpdatedFilesWithoutConfigChange, I changed the code to use a simple Java forEach rather than a Scala conversion, in order to feel more confident that concurrent modifications to the List would not have any bad effects here. (forEach is always safe on CopyOnWriteArrayList.) --- .../scala/kafka/server/DynamicBrokerConfig.scala | 26 + .../kafka/server/KRaftClusterTest.scala| 34 -- 2 files changed, 46 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index e2879d4378e..156d2d02b09 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -303,17 +303,17 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging addBrokerReconfigurable(controller.socketServer) } - def addReconfigurable(reconfigurable: Reconfigurable): Unit = CoreUtils.inWriteLock(lock) { + def addReconfigurable(reconfigurable: Reconfigurable): Unit = { verifyReconfigurableConfigs(reconfigurable.reconfigurableConfigs.asScala) reconfigurables.add(reconfigurable) } - def addBrokerReconfigurable(reconfigurable: BrokerReconfigurable): Unit = CoreUtils.inWriteLock(lock) { + def addBrokerReconfigurable(reconfigurable: BrokerReconfigurable): Unit = { verifyReconfigurableConfigs(reconfigurable.reconfigurableConfigs) brokerReconfigurables.add(reconfigurable) } - def removeReconfigurable(reconfigurable: Reconfigurable): Unit = CoreUtils.inWriteLock(lock) { + def removeReconfigurable(reconfigurable: Reconfigurable): Unit = { reconfigurables.remove(reconfigurable) } @@ -370,16 +370,18 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging * changes are processed. At the moment, only listener configs are considered for reloading. */ private[server] def reloadUpdatedFilesWithoutConfigChange(newProps: Properties): Unit = CoreUtils.inWriteLock(lock) { -reconfigurables.asScala - .filter(reconfigurable => ReloadableFileConfigs.exists(reconfigurable.reconfigurableConfigs.contains)) - .foreach { -case reconfigurable: ListenerReconfigurable => - val kafkaProps = validatedKafkaProps(newProps, perBrokerConfig = true) - val newConfig = new KafkaConfig(kafkaProps.asJava, false, None) - processListenerReconfigurable(reconfigurable, newConfig, Collections.emptyMap(), validateOnly = false, reloadOnly = true) -case reconfigurable => - trace(s"Files will not be reloaded without config change for $reconfigurable") +reconfigurables.forEach(r => { + if (ReloadableFileConfigs.exists(r.reconfigurableConfigs.contains)) { +r match { + case reconfigurable: ListenerReconfigurable => +val kafkaProps = validatedKafkaProps(newProps, perBrokerConfig = true) +val newConfig = new KafkaConfig(kafkaProps.asJava, false, None) +processListenerReconfigurable(reconfigurable, newConfig, Collections.emptyMap(), validateOnly = false, reloadOnly = true) + case reconfigurable => +trace(s"Files will not be reloaded without config change for $reconfigurable") +} } +}) } private def maybeCreatePasswordEncoder(secret: Option[Password]): Option[PasswordEncoder] = { diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index f5db6bd1a95..f54f4e8b3fd 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -46,7 +46,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapMetadata import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.quota import org.apache.k
(kafka) branch trunk updated: MINOR: fix typo in KAFKA-16515
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new bac8df56ffd MINOR: fix typo in KAFKA-16515 bac8df56ffd is described below commit bac8df56ffdf8a64ecfb78ec0779bcbc8e9f7c10 Author: Colin P. McCabe AuthorDate: Mon May 27 08:53:53 2024 -0700 MINOR: fix typo in KAFKA-16515 --- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 5531ab1dbc3..4702b7e5a97 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -441,7 +441,7 @@ class KafkaServer( CompletableFuture.completedFuture(quorumVoters), fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) - val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) + quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) val brokerToQuorumChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider = quorumControllerNodeProvider, time = time,
(kafka) branch trunk updated: KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 4f55786a8a8 KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers 4f55786a8a8 is described below commit 4f55786a8a86fe228a0b10a2f28529f5128e5d6f Author: Colin P. McCabe AuthorDate: Mon May 20 15:41:52 2024 -0700 KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers ZkMetadataCache could theoretically return KRaft controller information from a call to ZkMetadataCache.getAliveBrokerNode, which doesn't make sense. KRaft controllers are not part of the set of brokers. The only use-case for this functionality was in MetadataCacheControllerNodeProvider during ZK migration, where it allowed ZK brokers in migration mode to forward requests to kcontrollers when appropriate. This PR changes MetadataCacheControllerNodeProvider to simply delegate to quorumControllerNodeProvider in this case. Reviewers: José Armando García Sancio --- core/src/main/scala/kafka/server/KafkaServer.scala | 17 +++-- .../src/main/scala/kafka/server/MetadataCache.scala | 3 +-- .../server/NodeToControllerChannelManager.scala | 21 + .../kafka/server/metadata/ZkMetadataCache.scala | 9 + .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java | 2 +- .../jmh/metadata/MetadataRequestBenchmark.java | 2 +- .../apache/kafka/jmh/server/CheckpointBench.java| 2 +- .../kafka/jmh/server/PartitionCreationBench.java| 2 +- 8 files changed, 18 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 106fd442259..5531ab1dbc3 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -165,8 +165,10 @@ class KafkaServer( var kafkaScheduler: KafkaScheduler = _ - var kraftControllerNodes: Seq[Node] = _ @volatile var metadataCache: ZkMetadataCache = _ + + @volatile var quorumControllerNodeProvider: RaftControllerNodeProvider = _ + var quotaManagers: QuotaFactory.QuotaManagers = _ val zkClientConfig: ZKClientConfig = KafkaServer.zkClientConfigFromKafkaConfig(config) @@ -324,20 +326,13 @@ class KafkaServer( remoteLogManagerOpt = createRemoteLogManager() -if (config.migrationEnabled) { - kraftControllerNodes = QuorumConfig.voterConnectionsToNodes( -QuorumConfig.parseVoterConnections(config.quorumVoters) - ).asScala -} else { - kraftControllerNodes = Seq.empty -} metadataCache = MetadataCache.zkMetadataCache( config.brokerId, config.interBrokerProtocolVersion, brokerFeatures, - kraftControllerNodes, config.migrationEnabled) -val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config) +val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config, + () => Option(quorumControllerNodeProvider).map(_.getControllerInfo())) /* initialize feature change listener */ _featureChangeListener = new FinalizedFeatureChangeListener(metadataCache, _zkClient) @@ -1075,6 +1070,8 @@ class KafkaServer( } _brokerState = BrokerState.NOT_RUNNING +quorumControllerNodeProvider = null + startupComplete.set(false) isShuttingDown.set(false) CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics), this) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 015e46a7652..b8eda3fe4dc 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -116,10 +116,9 @@ object MetadataCache { def zkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures = BrokerFeatures.createEmpty(), - kraftControllerNodes: collection.Seq[Node] = collection.Seq.empty[Node], zkMigrationEnabled: Boolean = false) : ZkMetadataCache = { -new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, kraftControllerNodes, zkMigrationEnabled) +new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, zkMigrationEnabled) } def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = { diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 36997a4ea49..0017a5876af 100644 --- a/c
(kafka) branch trunk updated (2432a1866e7 -> 90892ae99fb)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git from 2432a1866e7 KAFKA-16373: KIP-1028: Adding code to support Apache Kafka Docker Official Images (#16027) add 90892ae99fb KAFKA-16516: Fix the controller node provider for broker to control channel No new revisions were added by this update. Summary of changes: core/src/main/scala/kafka/raft/RaftManager.scala | 13 +--- .../src/main/scala/kafka/server/BrokerServer.scala | 6 ++-- .../main/scala/kafka/server/ControllerServer.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 3 +- .../server/NodeToControllerChannelManager.scala| 8 ++--- .../kafka/server/KRaftClusterTest.scala| 38 ++ .../org/apache/kafka/raft/KafkaRaftClient.java | 5 +++ .../java/org/apache/kafka/raft/QuorumState.java| 1 + .../org/apache/kafka/raft/internals/VoterSet.java | 14 9 files changed, 74 insertions(+), 16 deletions(-)
(kafka) branch trunk updated (2c0b8b69207 -> 1fd39150aa3)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git from 2c0b8b69207 MINOR: ConsumerGroup#getOrMaybeCreateMember should not add the member to the group (#15847) add 1fd39150aa3 KAFKA-16655: Deflake ZKMigrationIntegrationTest.testDualWrite #15845 No new revisions were added by this update. Summary of changes: .../kafka/zk/ZkMigrationIntegrationTest.scala | 22 ++ 1 file changed, 18 insertions(+), 4 deletions(-)
(kafka) branch KAFKA-16624 deleted (was bdc07910b00)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16624 in repository https://gitbox.apache.org/repos/asf/kafka.git was bdc07910b00 New test, etc. The revisions that were on this branch are still contained in other references; therefore, this change does not discard any commits from the repository.
(kafka) branch trunk updated: KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new a3f24149905 KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810) a3f24149905 is described below commit a3f24149905886ac719901eeab8dc6d7b3c79614 Author: Colin Patrick McCabe AuthorDate: Thu May 2 09:23:25 2024 -0700 KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810) Fix a case where we could generate useless PartitionChangeRecords on metadata versions older than 3.6-IV0. This could happen in the case where we had an ISR with only one broker in it, and we were trying to go down to a fully empty ISR. In this case, PartitionChangeBuilder would block the record to going down to a fully empty ISR (since that is not valid in these pre-KIP-966 metadata versions), but it would still emit the record, even though it had no effect. Reviewers: Igor Soarez --- .../kafka/controller/PartitionChangeBuilder.java | 89 + .../controller/PartitionChangeBuilderTest.java | 213 - 2 files changed, 181 insertions(+), 121 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index 7f1b2cb6d17..0d2c1bd6a9d 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -214,6 +214,10 @@ public class PartitionChangeBuilder { } } +public List targetIsr() { +return targetIsr; +} + // VisibleForTesting /** * Perform leader election based on the partition state and leader election type. @@ -365,44 +369,61 @@ public class PartitionChangeBuilder { } /** - * Trigger a leader epoch bump if one is needed. - * - * We need to bump the leader epoch if: - * 1. The leader changed, or - * 2. The new replica list does not contain all the nodes that the old replica list did. + * Trigger a leader epoch bump if one is needed because of replica reassignment. * - * Changes that do NOT fall in any of these categories will increase the partition epoch, but - * not the leader epoch. Note that if the leader epoch increases, the partition epoch will - * always increase as well; there is no case where the partition epoch increases more slowly - * than the leader epoch. - * - * If the PartitionChangeRecord sets the leader field to something other than - * NO_LEADER_CHANGE, a leader epoch bump will automatically occur. That takes care of - * case 1. In this function, we check for cases 2 and 3, and handle them by manually - * setting record.leader to the current leader. - * - * In MV before 3.6 there was a bug (KAFKA-15021) in the brokers' replica manager - * that required that the leader epoch be bump whenever the ISR shrank. In MV 3.6 this leader - * bump is not required when the ISR shrinks. Note, that the leader epoch is never increased if - * the ISR expanded. + * Note that if the leader epoch increases, the partition epoch will always increase as well; there is no + * case where the partition epoch increases more slowly than the leader epoch. + */ +void triggerLeaderEpochBumpForReplicaReassignmentIfNeeded(PartitionChangeRecord record) { +if (record.leader() != NO_LEADER_CHANGE) { +// The leader is already changing, so there will already be a leader epoch bump. +return; +} +if (!Replicas.contains(targetReplicas, partition.replicas)) { +// If the new replica list does not contain all the brokers that the old one did, +// ensure that there will be a leader epoch bump by setting the leader field. +record.setLeader(partition.leader); +} +} + +/** + * Trigger a leader epoch bump if one is needed because of an ISR shrink. * - * In MV 3.6 and beyond, if the controller is in ZK migration mode, the leader epoch must - * be bumped during ISR shrink for compatability with ZK brokers. + * Note that it's important to call this function only after we have set the ISR field in + * the PartitionChangeRecord. */ -void triggerLeaderEpochBumpIfNeeded(PartitionChangeRecord record) { -if (record.leader() == NO_LEADER_CHANGE) { -boolean bumpLeaderEpochOnIsrShrink = metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled; - -if (!Replicas.contains(targetReplicas, partition.replicas)) { -// Reassignment -record.setLeader(partition.leader
(kafka) branch KAFKA-16624 updated (65d96aeaf50 -> bdc07910b00)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16624 in repository https://gitbox.apache.org/repos/asf/kafka.git from 65d96aeaf50 KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV add bdc07910b00 New test, etc. No new revisions were added by this update. Summary of changes: .../kafka/controller/PartitionChangeBuilder.java | 5 ++-- .../controller/PartitionChangeBuilderTest.java | 33 ++ 2 files changed, 35 insertions(+), 3 deletions(-)
(kafka) 01/01: Fixes
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16649 in repository https://gitbox.apache.org/repos/asf/kafka.git commit f435287ac6315f9bacd41c7862559b42cbd8f395 Author: Colin P. McCabe AuthorDate: Wed May 1 14:25:18 2024 -0700 Fixes --- core/src/main/scala/kafka/server/DynamicBrokerConfig.scala| 8 +++- .../test/scala/integration/kafka/server/KRaftClusterTest.scala| 4 ++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 924bb0f660e..d2d518f1dd2 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -221,7 +221,13 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging // collections, while another thread is iterating over them. private[server] val reconfigurables = new CopyOnWriteArrayList[Reconfigurable]() private val brokerReconfigurables = new CopyOnWriteArrayList[BrokerReconfigurable]() + + /** + * The DynamicBrokerConfig lock which prevents concurrent changes to dynamic configuration. + * (It does not prevent new reconfigurables from being registered or unregistered, though.) + */ private val lock = new ReentrantReadWriteLock + private var metricsReceiverPluginOpt: Option[ClientMetricsReceiverPlugin] = _ private var currentConfig: KafkaConfig = _ private val dynamicConfigPasswordEncoder = if (kafkaConfig.processRoles.isEmpty) { @@ -317,7 +323,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging reconfigurables.remove(reconfigurable) } - private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = CoreUtils.inWriteLock(lock) { + private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = { val nonDynamic = configNames.filter(DynamicConfig.Broker.nonDynamicProps.contains) require(nonDynamic.isEmpty, s"Reconfigurable contains non-dynamic configs $nonDynamic") } diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 83182d54443..d59d8859562 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -1548,7 +1548,7 @@ class KRaftClusterTest { new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(1).build()). - setConfigProp(KafkaConfig.NumNetworkThreadsProp, "4"). + setConfigProp(KafkaConfig.NumNetworkThreadsProp, "2"). build() try { cluster.format() @@ -1559,7 +1559,7 @@ class KRaftClusterTest { admin.incrementalAlterConfigs( Collections.singletonMap(new ConfigResource(Type.BROKER, ""), Collections.singletonList(new AlterConfigOp( - new ConfigEntry(KafkaConfig.NumNetworkThreadsProp, "2"), OpType.SET.all().get() + new ConfigEntry(KafkaConfig.NumNetworkThreadsProp, "3"), OpType.SET.all().get() val newTopic = Collections.singletonList(new NewTopic("test-topic", 1, 1.toShort)) val createTopicResult = admin.createTopics(newTopic) createTopicResult.all().get()
(kafka) branch KAFKA-16649 created (now f435287ac63)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16649 in repository https://gitbox.apache.org/repos/asf/kafka.git at f435287ac63 Fixes This branch includes the following new commits: new f435287ac63 Fixes The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) branch trunk updated: KAFKA-16475: add more tests to TopicImageNodeTest (#15735)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 31355ef8f94 KAFKA-16475: add more tests to TopicImageNodeTest (#15735) 31355ef8f94 is described below commit 31355ef8f948f369e240ebc203f889f187116d75 Author: mannoopj <139923522+manno...@users.noreply.github.com> AuthorDate: Tue Apr 30 17:59:00 2024 -0400 KAFKA-16475: add more tests to TopicImageNodeTest (#15735) Add more test cases to TopicImageNodeTest.java. Reviewers: Colin P. McCabe --- .../kafka/image/node/TopicImageNodeTest.java | 43 +++--- 1 file changed, 38 insertions(+), 5 deletions(-) diff --git a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java index 83855d94b8c..8ecd4be28c9 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java @@ -17,14 +17,17 @@ package org.apache.kafka.image.node; +import org.apache.kafka.common.DirectoryId; import org.apache.kafka.common.Uuid; import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.node.printer.NodeStringifier; +import org.apache.kafka.metadata.LeaderRecoveryState; +import org.apache.kafka.metadata.PartitionRegistration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; - import java.util.Arrays; -import java.util.Collections; - +import java.util.HashMap; +import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -32,11 +35,21 @@ import static org.junit.jupiter.api.Assertions.assertNull; @Timeout(value = 40) public class TopicImageNodeTest { -private final static TopicImageNode NODE = new TopicImageNode(new TopicImage("topic-image-node-test-topic", Uuid.ZERO_UUID, Collections.emptyMap())); +private final static TopicImageNode NODE = new TopicImageNode(newTopicImage("topic-image-node-test-topic", Uuid.ZERO_UUID, new PartitionRegistration.Builder().setReplicas(new int[] {2, 3, 4}). +setDirectories(DirectoryId.migratingArray(3)). +setIsr(new int[] {2, 3}).setLeader(2).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(1).setPartitionEpoch(345).build())); +private static TopicImage newTopicImage(String name, Uuid id, PartitionRegistration... partitions) { +Map partitionMap = new HashMap<>(); +int i = 0; +for (PartitionRegistration partition : partitions) { +partitionMap.put(i++, partition); +} +return new TopicImage(name, id, partitionMap); +} @Test public void testChildNames() { -assertEquals(Arrays.asList("name", "id"), NODE.childNames()); +assertEquals(Arrays.asList("name", "id", "0"), NODE.childNames()); } @Test @@ -57,4 +70,24 @@ public class TopicImageNodeTest { public void testUnknownChild() { assertNull(NODE.child("unknown")); } + +@Test +public void testChildPartitionId() { +MetadataNode child = NODE.child("0"); +assertNotNull(child); +NodeStringifier stringifier = new NodeStringifier(); +child.print(stringifier); +assertEquals("PartitionRegistration(replicas=[2, 3, 4], " + +"directories=[AA, AA, AA], " + +"isr=[2, 3], removingReplicas=[], addingReplicas=[], elr=[], lastKnownElr=[], leader=2, " + +"leaderRecoveryState=RECOVERED, leaderEpoch=1, partitionEpoch=345)", stringifier.toString()); +} + +@Test +public void testChildPartitionIdNull() { +MetadataNode child1 = NODE.child("1"); +MetadataNode child2 = NODE.child("a"); +assertNull(child1); +assertNull(child2); +} }
(kafka) branch trunk updated: MINOR: Change the documentation of the Brokers field. (#15809)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new ba6a73776b6 MINOR: Change the documentation of the Brokers field. (#15809) ba6a73776b6 is described below commit ba6a73776b65603b4c866f8d2db966d826031e74 Author: Emanuele Sabellico AuthorDate: Sat Apr 27 00:53:18 2024 +0200 MINOR: Change the documentation of the Brokers field. (#15809) Change the documentation of the Brokers field to make it clear that it doesn't always have all the brokers that are listed as replicas. Reviewer: Colin P. McCabe --- clients/src/main/resources/common/message/MetadataResponse.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/resources/common/message/MetadataResponse.json b/clients/src/main/resources/common/message/MetadataResponse.json index bb0b6a802c3..408cdc7940a 100644 --- a/clients/src/main/resources/common/message/MetadataResponse.json +++ b/clients/src/main/resources/common/message/MetadataResponse.json @@ -48,7 +48,7 @@ { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, { "name": "Brokers", "type": "[]MetadataResponseBroker", "versions": "0+", - "about": "Each broker in the response.", "fields": [ + "about": "A list of brokers present in the cluster.", "fields": [ { "name": "NodeId", "type": "int32", "versions": "0+", "mapKey": true, "entityType": "brokerId", "about": "The broker ID." }, { "name": "Host", "type": "string", "versions": "0+",
(kafka) branch KAFKA-16624 updated: KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16624 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/KAFKA-16624 by this push: new 65d96aeaf50 KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV 65d96aeaf50 is described below commit 65d96aeaf50b6076b982b4b2e69fec0803c4be83 Author: Colin P. McCabe AuthorDate: Thu Apr 25 14:08:25 2024 -0700 KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV Fix a case where we could generate useless PartitionChangeRecords on metadata versions older than 3.6-IV0. This could happen in the case where we had an ISR with only one broker in it, and we were trying to go down to a fully empty ISR. In this case, PartitionChangeBuilder would block the record to going down to a fully empty ISR (since that is not valid in these pre-KIP-966 metadata versions), but it would still emit the record, even though it had no effect. --- .../kafka/controller/PartitionChangeBuilder.java | 94 +++ .../controller/PartitionChangeBuilderTest.java | 180 +++-- 2 files changed, 151 insertions(+), 123 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index 7f1b2cb6d17..59733445695 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -214,6 +214,10 @@ public class PartitionChangeBuilder { } } +public List targetIsr() { +return targetIsr; +} + // VisibleForTesting /** * Perform leader election based on the partition state and leader election type. @@ -365,45 +369,62 @@ public class PartitionChangeBuilder { } /** - * Trigger a leader epoch bump if one is needed. - * - * We need to bump the leader epoch if: - * 1. The leader changed, or - * 2. The new replica list does not contain all the nodes that the old replica list did. - * - * Changes that do NOT fall in any of these categories will increase the partition epoch, but - * not the leader epoch. Note that if the leader epoch increases, the partition epoch will - * always increase as well; there is no case where the partition epoch increases more slowly - * than the leader epoch. + * Trigger a leader epoch bump if one is needed because of replica reassignment. * - * If the PartitionChangeRecord sets the leader field to something other than - * NO_LEADER_CHANGE, a leader epoch bump will automatically occur. That takes care of - * case 1. In this function, we check for cases 2 and 3, and handle them by manually - * setting record.leader to the current leader. - * - * In MV before 3.6 there was a bug (KAFKA-15021) in the brokers' replica manager - * that required that the leader epoch be bump whenever the ISR shrank. In MV 3.6 this leader - * bump is not required when the ISR shrinks. Note, that the leader epoch is never increased if - * the ISR expanded. + * Note that if the leader epoch increases, the partition epoch will always increase as well; there is no + * case where the partition epoch increases more slowly than the leader epoch. + */ +void triggerLeaderEpochBumpForReplicaReassignmentIfNeeded(PartitionChangeRecord record) { +if (record.leader() != NO_LEADER_CHANGE) { +// The leader is already changing, so there will already be a leader epoch bump. +return; +} +if (!Replicas.contains(targetReplicas, partition.replicas)) { +// If the new replica list does not contain all the brokers that the old one did, +// ensure that there will be a leader epoch bump by setting the leader field. +record.setLeader(partition.leader); +} +} + +/** + * Trigger a leader epoch bump if one is needed because of an ISR shrink. * - * In MV 3.6 and beyond, if the controller is in ZK migration mode, the leader epoch must - * be bumped during ISR shrink for compatability with ZK brokers. + * Note that it's important to call this function only after we have set the ISR field in + * the PartitionChangeRecord. */ -void triggerLeaderEpochBumpIfNeeded(PartitionChangeRecord record) { -if (record.leader() == NO_LEADER_CHANGE) { -boolean bumpLeaderEpochOnIsrShrink = metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled; - -if (!Replicas.contains(targetReplicas, partition.replicas)) { -// Reassignment -record.setLeader(partition.leader); -} else if (bumpLeaderEpochOnIsrShrink
(kafka) branch KAFKA-16624 created (now 6feae817d25)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16624 in repository https://gitbox.apache.org/repos/asf/kafka.git at 6feae817d25 MINOR: Rename RaftConfig to QuorumConfig (#15797) No new revisions were added by this update.
(kafka) branch 3.7 updated: KAFKA-16003: Always create the /config/topics ZNode even for topics without configs (#15022)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new f6bf85edbfb KAFKA-16003: Always create the /config/topics ZNode even for topics without configs (#15022) f6bf85edbfb is described below commit f6bf85edbfb0885ce192fae19b3b7866e687c886 Author: Mickael Maison AuthorDate: Thu Jan 25 15:46:24 2024 +0100 KAFKA-16003: Always create the /config/topics ZNode even for topics without configs (#15022) Reviewers: Luke Chen Conflicts: Handle the fact that the ConfigType refactor is missing from 3.7. --- .../zk/migration/ZkTopicMigrationClient.scala | 11 ++- .../kafka/zk/ZkMigrationIntegrationTest.scala | 104 +++-- 2 files changed, 87 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala index dd042ff96a7..6cd752d2d21 100644 --- a/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala @@ -34,6 +34,7 @@ import org.apache.zookeeper.CreateMode import org.apache.zookeeper.KeeperException.Code import java.util +import java.util.Properties import scala.collection.Seq import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ @@ -122,9 +123,17 @@ class ZkTopicMigrationClient(zkClient: KafkaZkClient) extends TopicMigrationClie zkClient.defaultAcls(path), CreateMode.PERSISTENT) } +val topicConfigZNode = { + val path = ConfigEntityZNode.path(ConfigType.Topic, topicName) + CreateRequest( +path, +ConfigEntityZNode.encode(new Properties()), +zkClient.defaultAcls(path), +CreateMode.PERSISTENT) +} val createPartitionZNodeReqs = createTopicPartitionZNodesRequests(topicName, partitions, state) -val requests = Seq(createTopicZNode) ++ createPartitionZNodeReqs +val requests = Seq(createTopicZNode, topicConfigZNode) ++ createPartitionZNodeReqs val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state) val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap if (resultCodes(TopicZNode.path(topicName)).equals(Code.NODEEXISTS)) { diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 15bb563ad1c..75eab542031 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -50,7 +50,7 @@ import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion, Pr import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue, fail} import org.junit.jupiter.api.{Assumptions, Timeout} import org.junit.jupiter.api.extension.ExtendWith -import org.slf4j.LoggerFactory +import org.slf4j.{Logger, LoggerFactory} import java.util import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit} @@ -91,7 +91,7 @@ object ZkMigrationIntegrationTest { @Timeout(300) class ZkMigrationIntegrationTest { - val log = LoggerFactory.getLogger(classOf[ZkMigrationIntegrationTest]) + val log: Logger = LoggerFactory.getLogger(classOf[ZkMigrationIntegrationTest]) class MetadataDeltaVerifier { val metadataDelta = new MetadataDelta(MetadataImage.EMPTY) @@ -184,7 +184,7 @@ class ZkMigrationIntegrationTest { // Enable migration configs and restart brokers log.info("Restart brokers in migration mode") zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") - zkCluster.config().serverProperties().put(RaftConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()); + zkCluster.config().serverProperties().put(RaftConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KafkaConfig.ListenerSecurityProtocolMapProp, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.rollingBrokerRestart() // This would throw if authorizers weren't allowed @@ -401,8 +401,8 @@ class ZkMigrationIntegrationTest { assertEquals(10, topicDescriptions("test-topic-3").partitions().size()) topicDescriptions.foreach { case (topic, description) => description.partitions().forEach(partition => { -assertEquals(3, partition.replicas().size(), s"Unexpected number of replica
(kafka) branch 3.7 updated: MINOR: Fix typo in javadoc (#15031)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 2c3674dbf83 MINOR: Fix typo in javadoc (#15031) 2c3674dbf83 is described below commit 2c3674dbf83cc85b6935b99398943ce72e6158b8 Author: jiangyuan AuthorDate: Tue Dec 19 00:35:13 2023 +0800 MINOR: Fix typo in javadoc (#15031) Reviewers: Divij Vaidya --- .../src/main/java/org/apache/kafka/message/MessageClassGenerator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java b/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java index 83117560a80..9b5f730b8ff 100644 --- a/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java @@ -27,7 +27,7 @@ public interface MessageClassGenerator { String outputName(MessageSpec spec); /** - * Generate the convertere, and then write it out. + * Generate the converter, and then write it out. * * @param spec The message to generate a converter for. * @param writerThe writer to write out the state to.
(kafka) branch 3.7 updated: MINOR: Fixed typo in quickstart documentation (#15037)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 5b64c599119 MINOR: Fixed typo in quickstart documentation (#15037) 5b64c599119 is described below commit 5b64c5991199db380ead1f6d149c4f48098f8082 Author: Lokesh Kumar AuthorDate: Mon Dec 18 13:51:36 2023 +0100 MINOR: Fixed typo in quickstart documentation (#15037) Reviewers: Divij Vaidya --- docs/quickstart.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quickstart.html b/docs/quickstart.html index 94f278b2b86..e0c84279347 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -47,7 +47,7 @@ $ cd kafka_{{scalaVersion}}-{{fullDotVersion}} -Apache Kafka can be started using ZooKeeper or KRaft. To get started with either configuration follow one the sections below but not both. +Apache Kafka can be started using ZooKeeper or KRaft. To get started with either configuration follow one of the sections below but not both.
(kafka) branch trunk updated: KAFKA-16475: add test for TopicImageNodeTest (#15720)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 5193eb93237 KAFKA-16475: add test for TopicImageNodeTest (#15720) 5193eb93237 is described below commit 5193eb93237ba9093ae444d73a1eaa2d6abcc9c1 Author: Johnny Hsu <44309740+johnnych...@users.noreply.github.com> AuthorDate: Wed Apr 17 01:20:34 2024 +0800 KAFKA-16475: add test for TopicImageNodeTest (#15720) Add a unit test for TopicImageNodeTest. Co-authored-by: Johnny Hsu Reviewers: Colin P. McCabe --- .../kafka/image/node/TopicImageNodeTest.java | 60 ++ 1 file changed, 60 insertions(+) diff --git a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java new file mode 100644 index 000..83855d94b8c --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.image.node; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.TopicImage; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + + +@Timeout(value = 40) +public class TopicImageNodeTest { +private final static TopicImageNode NODE = new TopicImageNode(new TopicImage("topic-image-node-test-topic", Uuid.ZERO_UUID, Collections.emptyMap())); + +@Test +public void testChildNames() { +assertEquals(Arrays.asList("name", "id"), NODE.childNames()); +} + +@Test +public void testNameChild() { +MetadataNode child = NODE.child("name"); +assertNotNull(child); +assertEquals(MetadataLeafNode.class, child.getClass()); +} + +@Test +public void testIdChild() { +MetadataNode child = NODE.child("id"); +assertNotNull(child); +assertEquals(MetadataLeafNode.class, child.getClass()); +} + +@Test +public void testUnknownChild() { +assertNull(NODE.child("unknown")); +} +}
(kafka) branch 3.7 updated: KAFKA-16286; Notify listener of latest leader and epoch (#15397)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 736fd76a12b KAFKA-16286; Notify listener of latest leader and epoch (#15397) 736fd76a12b is described below commit 736fd76a12b173f952794356791d0b06ed15c127 Author: José Armando García Sancio AuthorDate: Fri Feb 23 12:56:25 2024 -0800 KAFKA-16286; Notify listener of latest leader and epoch (#15397) KRaft was only notifying listeners of the latest leader and epoch when the replica transition to a new state. This can result in the listener never getting notified if the registration happened after it had become a follower. This problem doesn't exists for the active leader because the KRaft implementation attempts to notified the listener of the latest leader and epoch when the replica is the active leader. This issue is fixed by notifying the listeners of the latest leader and epoch after processing the listener registration request. Reviewers: Colin P. McCabe --- .../org/apache/kafka/raft/KafkaRaftClient.java | 8 +++ .../org/apache/kafka/raft/KafkaRaftClientTest.java | 60 ++ .../apache/kafka/raft/RaftClientTestContext.java | 6 +++ 3 files changed, 74 insertions(+) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index e9fca1d6b2b..8790de92405 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -2202,6 +2202,14 @@ public class KafkaRaftClient implements RaftClient { quorum.highWatermark().ifPresent(highWatermarkMetadata -> { updateListenersProgress(highWatermarkMetadata.offset); }); + +// Notify the new listeners of the latest leader and epoch +Optional> leaderState = quorum.maybeLeaderState(); +if (leaderState.isPresent()) { +maybeFireLeaderChange(leaderState.get()); +} else { +maybeFireLeaderChange(); +} } private void processRegistration(Registration registration) { diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 51a5938220f..f18a0ff2e88 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -2958,6 +2958,66 @@ public class KafkaRaftClientTest { assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } +@Test +public void testHandleLeaderChangeFiresAfterUnattachedRegistration() throws Exception { +// When registering a listener while the replica is unattached, it should get notified +// with the current epoch +// When transitioning to follower, expect another notification with the leader and epoch + +int localId = 0; +int otherNodeId = 1; +int epoch = 7; +Set voters = Utils.mkSet(localId, otherNodeId); + +RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) +.withUnknownLeader(epoch) +.build(); + +// Register another listener and verify that it is notified of latest epoch +RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( +OptionalInt.of(localId) +); +context.client.register(secondListener); +context.client.poll(); + +// Expected leader change notification +LeaderAndEpoch expectedLeaderAndEpoch = new LeaderAndEpoch(OptionalInt.empty(), epoch); +assertEquals(expectedLeaderAndEpoch, secondListener.currentLeaderAndEpoch()); + +// Transition to follower and the expect a leader changed notification +context.deliverRequest(context.beginEpochRequest(epoch, otherNodeId)); +context.pollUntilResponse(); + +// Expected leader change notification +expectedLeaderAndEpoch = new LeaderAndEpoch(OptionalInt.of(otherNodeId), epoch); +assertEquals(expectedLeaderAndEpoch, secondListener.currentLeaderAndEpoch()); +} + +@Test +public void testHandleLeaderChangeFiresAfterFollowerRegistration() throws Exception { +// When registering a listener while the replica is a follower, it should get notified with +// the current leader and epoch + +int localId = 0; +int otherNodeId = 1; +int epoch = 7; +Set voters = Utils.mkSet(localId, otherNodeId); + +RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) +.withElectedLeader(epoch, otherNodeId) +
(kafka) branch 3.7 updated: KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 6baa6deea5b KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695) 6baa6deea5b is described below commit 6baa6deea5bdd6aafd83131c6ed9c5c968b09c29 Author: Colin Patrick McCabe AuthorDate: Thu Apr 11 09:34:27 2024 -0700 KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695) The CurrentControllerId metric added by KIP-1001 is unreliable in ZK mode. Sometimes when there is no active ZK-based controller, it still shows the previous controller ID. Instead, it should show -1 in that situation. This PR fixes that by using the controller ID from the KafkaController.scala, which is obtained directly from the controller znode. It also adds a new test, ControllerIdMetricTest.scala. Reviewers: David Arthur --- .../scala/kafka/controller/KafkaController.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 24 +- .../unit/kafka/server/ControllerIdMetricTest.scala | 52 ++ 3 files changed, 67 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b989837f891..15a22acae6e 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -161,7 +161,7 @@ class KafkaController(val config: KafkaConfig, private val isrChangeNotificationHandler = new IsrChangeNotificationHandler(eventManager) private val logDirEventNotificationHandler = new LogDirEventNotificationHandler(eventManager) - @volatile private var activeControllerId = -1 + @volatile var activeControllerId = -1 @volatile private var offlinePartitionCount = 0 @volatile private var preferredReplicaImbalanceCount = 0 @volatile private var globalTopicCount = 0 diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 203d2fffd51..874022d3534 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -195,7 +195,7 @@ class KafkaServer( override def logManager: LogManager = _logManager - def kafkaController: KafkaController = _kafkaController + @volatile def kafkaController: KafkaController = _kafkaController var lifecycleManager: BrokerLifecycleManager = _ private var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _ @@ -657,15 +657,19 @@ class KafkaServer( } def createCurrentControllerIdMetric(): Unit = { - KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID, () => { - Option(metadataCache) match { -case None => -1 -case Some(cache) => cache.getControllerId match { - case None => -1 - case Some(id) => id.id -} - } -}) + KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID, + () => getCurrentControllerIdFromOldController()) + } + + /** + * Get the current controller ID from the old controller code. + * This is the most up-to-date controller ID we can get when in ZK mode. + */ + def getCurrentControllerIdFromOldController(): Int = { +Option(_kafkaController) match { + case None => -1 + case Some(controller) => controller.activeControllerId +} } def unregisterCurrentControllerIdMetric(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala new file mode 100755 index 000..8d0328f02a4 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.integration.KafkaServerTestHarness +import kafka.utils.
(kafka) branch 3.7 updated: KAFKA-16463 Delete metadata log on ZK broker startup (#15648)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 7bb8324121c KAFKA-16463 Delete metadata log on ZK broker startup (#15648) 7bb8324121c is described below commit 7bb8324121c2036282a383b11a50abb2426cc939 Author: David Arthur AuthorDate: Fri Apr 12 13:21:30 2024 -0400 KAFKA-16463 Delete metadata log on ZK broker startup (#15648) This patch changes the behavior of the migrating ZK broker to always delete the local metadata log during startup. This deletion is done immediately before creating the RaftManager which will re-create the log directory and let the broker re-replicate the log from the active controller. This new behavior is only present for ZK brokers that having migrations enabled. KRaft brokers, even those with migrations enabled, will not delete their local metadata log. KRaft controllers are not impacted by this change. The rationale for this change is to make it easier for operators to re-attempt a ZK to KRaft migration after having reverted back to ZK mode. If an operator has reverted back to ZK mode, there will be an invalid metadata log on the disk of each broker. In order to re-attempt the migration in the future, this log needs to be deleted. This can be pretty burdensome to the operator for large clusters, especially since the log deletion must be done while the broker is offline. Reviewers: Colin P. McCabe , Igor Soarez , Chia-Ping Tsai Conflicts: Fix some conflicts in ZkMigrationIntegrationTest.scala related to refactorings that happened in master but not 3.7, such as the ProcessRoles class getting split out. --- core/src/main/scala/kafka/raft/RaftManager.scala | 54 ++- core/src/main/scala/kafka/server/KafkaServer.scala | 7 +- .../server/NodeToControllerChannelManager.scala| 7 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 83 +- .../scala/unit/kafka/raft/RaftManagerTest.scala| 179 +++-- 5 files changed, 270 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index cd8a9739654..9c56dc35416 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -34,13 +34,14 @@ import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClie import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid +import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} import org.apache.kafka.common.protocol.ApiMessage import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.common.utils.{LogContext, Time, Utils} import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, NON_ROUTABLE_ADDRESS, UnknownAddressSpec} import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, KafkaRaftClient, LeaderAndEpoch, RaftClient, RaftConfig, RaftRequest, ReplicatedLog} import org.apache.kafka.server.common.serialization.RecordSerde @@ -107,6 +108,51 @@ object KafkaRaftManager { lock } + + /** + * Test if the configured metadata log dir is one of the data log dirs. + */ + def hasDifferentLogDir(config: KafkaConfig): Boolean = { +!config + .logDirs + .map(Paths.get(_).toAbsolutePath) + .contains(Paths.get(config.metadataLogDir).toAbsolutePath) + } + + /** + * Obtain the file lock and delete the metadata log directory completely. + * + * This is only used by ZK brokers that are in pre-migration or hybrid mode of the ZK to KRaft migration. + * The rationale for deleting the metadata log in these cases is that it is safe to do on brokers and it + * it makes recovery from a failed migration much easier. See KAFKA-16463. + * + * @param config The broker config + */ + def maybeDeleteMetadataLogDir(config: KafkaConfig): Unit = { +// These constraints are enforced in KafkaServer, but repeating them here to guard against future callers +if (config.processRoles.nonEmpty) { + throw new RuntimeException("Not deleting metadata log dir since this node is in KRaft mode.") +} else if (!config.migrationEnabled) { + throw new RuntimeException("Not deleting metadata log dir since migrations are not enabled.") +} else { + val metadataDir = new File(config.metadataLogDir) +
(kafka) branch trunk updated: KAFKA-16463 Delete metadata log on ZK broker startup (#15648)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new e02ffd852fa KAFKA-16463 Delete metadata log on ZK broker startup (#15648) e02ffd852fa is described below commit e02ffd852fae7c1d2681621be7eb888e3805e027 Author: David Arthur AuthorDate: Fri Apr 12 13:21:30 2024 -0400 KAFKA-16463 Delete metadata log on ZK broker startup (#15648) This patch changes the behavior of the migrating ZK broker to always delete the local metadata log during startup. This deletion is done immediately before creating the RaftManager which will re-create the log directory and let the broker re-replicate the log from the active controller. This new behavior is only present for ZK brokers that having migrations enabled. KRaft brokers, even those with migrations enabled, will not delete their local metadata log. KRaft controllers are not impacted by this change. The rationale for this change is to make it easier for operators to re-attempt a ZK to KRaft migration after having reverted back to ZK mode. If an operator has reverted back to ZK mode, there will be an invalid metadata log on the disk of each broker. In order to re-attempt the migration in the future, this log needs to be deleted. This can be pretty burdensome to the operator for large clusters, especially since the log deletion must be done while the broker is offline. Reviewers: Colin P. McCabe , Igor Soarez , Chia-Ping Tsai --- core/src/main/scala/kafka/raft/RaftManager.scala | 54 +++- core/src/main/scala/kafka/server/KafkaServer.scala | 7 +- .../server/NodeToControllerChannelManager.scala| 7 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 83 +++- .../scala/unit/kafka/raft/RaftManagerTest.scala| 144 +++-- 5 files changed, 277 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index a9e64fb967b..a16e528d797 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -32,13 +32,14 @@ import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClie import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid +import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} import org.apache.kafka.common.protocol.ApiMessage import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.common.utils.{LogContext, Time, Utils} import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, NON_ROUTABLE_ADDRESS, UnknownAddressSpec} import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, RaftConfig, ReplicatedLog} import org.apache.kafka.server.ProcessRole @@ -69,6 +70,51 @@ object KafkaRaftManager { lock } + + /** + * Test if the configured metadata log dir is one of the data log dirs. + */ + def hasDifferentLogDir(config: KafkaConfig): Boolean = { +!config + .logDirs + .map(Paths.get(_).toAbsolutePath) + .contains(Paths.get(config.metadataLogDir).toAbsolutePath) + } + + /** + * Obtain the file lock and delete the metadata log directory completely. + * + * This is only used by ZK brokers that are in pre-migration or hybrid mode of the ZK to KRaft migration. + * The rationale for deleting the metadata log in these cases is that it is safe to do on brokers and it + * it makes recovery from a failed migration much easier. See KAFKA-16463. + * + * @param config The broker config + */ + def maybeDeleteMetadataLogDir(config: KafkaConfig): Unit = { +// These constraints are enforced in KafkaServer, but repeating them here to guard against future callers +if (config.processRoles.nonEmpty) { + throw new RuntimeException("Not deleting metadata log dir since this node is in KRaft mode.") +} else if (!config.migrationEnabled) { + throw new RuntimeException("Not deleting metadata log dir since migrations are not enabled.") +} else { + val metadataDir = new File(config.metadataLogDir) + val logDirName = UnifiedLog.logDirName(Topic.CLUSTER_METADATA_TOPIC_PARTITION) + val metadataPartitionDir = KafkaRaftManager.createLogDirectory(metadataDir, logDirName) +
(kafka) branch trunk updated: KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new b67a3fa79da KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695) b67a3fa79da is described below commit b67a3fa79dab8bb253be353fb35d9e9f8536d750 Author: Colin Patrick McCabe AuthorDate: Thu Apr 11 09:34:27 2024 -0700 KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695) The CurrentControllerId metric added by KIP-1001 is unreliable in ZK mode. Sometimes when there is no active ZK-based controller, it still shows the previous controller ID. Instead, it should show -1 in that situation. This PR fixes that by using the controller ID from the KafkaController.scala, which is obtained directly from the controller znode. It also adds a new test, ControllerIdMetricTest.scala. Reviewers: David Arthur --- .../scala/kafka/controller/KafkaController.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 24 +- .../unit/kafka/server/ControllerIdMetricTest.scala | 52 ++ 3 files changed, 67 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 5b893c8e851..62887395e32 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -161,7 +161,7 @@ class KafkaController(val config: KafkaConfig, private val isrChangeNotificationHandler = new IsrChangeNotificationHandler(eventManager) private val logDirEventNotificationHandler = new LogDirEventNotificationHandler(eventManager) - @volatile private var activeControllerId = -1 + @volatile var activeControllerId = -1 @volatile private var offlinePartitionCount = 0 @volatile private var preferredReplicaImbalanceCount = 0 @volatile private var globalTopicCount = 0 diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 369bae7e4cb..b8da5f964de 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -197,7 +197,7 @@ class KafkaServer( override def logManager: LogManager = _logManager - def kafkaController: KafkaController = _kafkaController + @volatile def kafkaController: KafkaController = _kafkaController var lifecycleManager: BrokerLifecycleManager = _ private var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _ @@ -657,15 +657,19 @@ class KafkaServer( } private def createCurrentControllerIdMetric(): Unit = { - KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID, () => { - Option(metadataCache) match { -case None => -1 -case Some(cache) => cache.getControllerId match { - case None => -1 - case Some(id) => id.id -} - } -}) + KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID, + () => getCurrentControllerIdFromOldController()) + } + + /** + * Get the current controller ID from the old controller code. + * This is the most up-to-date controller ID we can get when in ZK mode. + */ + def getCurrentControllerIdFromOldController(): Int = { +Option(_kafkaController) match { + case None => -1 + case Some(controller) => controller.activeControllerId +} } private def unregisterCurrentControllerIdMetric(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala new file mode 100755 index 000..8d0328f02a4 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.integration.KafkaServerTestHarness +import kafk
(kafka) branch 3.6 updated: KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new 7303b6063bc KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608) 7303b6063bc is described below commit 7303b6063bc16a4bc8a75083a55460be08c5c9c0 Author: Colin Patrick McCabe AuthorDate: Wed Mar 27 12:26:36 2024 -0700 KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608) Reviewers: Chia-Ping Tsai , Luke Chen --- core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala index 844c1aabc4c..fbf6d7c3434 100644 --- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala @@ -244,7 +244,7 @@ class ZkConfigMigrationClient( state } else if (responses.head.resultCode.equals(Code.OK)) { // Write the notification znode if our update was successful -zkClient.createConfigChangeNotification(s"$configType/$configName") + zkClient.createConfigChangeNotification(s"${configType.get}/$configName") state.withMigrationZkVersion(migrationZkVersion) } else { throw KeeperException.create(responses.head.resultCode, path)
(kafka) branch 3.7 updated: KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 313f3f22b38 KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608) 313f3f22b38 is described below commit 313f3f22b38edafaf87f1e68b16b22fd8c41808c Author: Colin Patrick McCabe AuthorDate: Wed Mar 27 12:26:36 2024 -0700 KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608) Reviewers: Chia-Ping Tsai , Luke Chen --- core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala index 844c1aabc4c..fbf6d7c3434 100644 --- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala @@ -244,7 +244,7 @@ class ZkConfigMigrationClient( state } else if (responses.head.resultCode.equals(Code.OK)) { // Write the notification znode if our update was successful -zkClient.createConfigChangeNotification(s"$configType/$configName") + zkClient.createConfigChangeNotification(s"${configType.get}/$configName") state.withMigrationZkVersion(migrationZkVersion) } else { throw KeeperException.create(responses.head.resultCode, path)
(kafka) branch trunk updated: KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new f40c06690bd KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608) f40c06690bd is described below commit f40c06690bd3d982ed704411b952e5f9645d5f76 Author: Colin Patrick McCabe AuthorDate: Wed Mar 27 12:26:36 2024 -0700 KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608) Reviewers: Chia-Ping Tsai , Luke Chen --- core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala index fda6e93c249..a1f6e1a112f 100644 --- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala @@ -244,7 +244,7 @@ class ZkConfigMigrationClient( state } else if (responses.head.resultCode.equals(Code.OK)) { // Write the notification znode if our update was successful -zkClient.createConfigChangeNotification(s"$configType/$configName") + zkClient.createConfigChangeNotification(s"${configType.get}/$configName") state.withMigrationZkVersion(migrationZkVersion) } else { throw KeeperException.create(responses.head.resultCode, path)
(kafka) branch 3.6 updated: KAFKA-16411: Correctly migrate default client quota entities (#15584)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new fbff947d259 KAFKA-16411: Correctly migrate default client quota entities (#15584) fbff947d259 is described below commit fbff947d25956100b1189fa4ee4ca64ea775df11 Author: Colin P. McCabe AuthorDate: Wed Mar 27 09:38:11 2024 -0700 KAFKA-16411: Correctly migrate default client quota entities (#15584) KAFKA-16222 fixed a bug whereby we didn't undo the name sanitization used on client quota entity names stored in ZooKeeper. However, it incorrectly claimed to fix the handling of default client quota entities. It also failed to correctly re-sanitize when syncronizing the data back to ZooKeeper. This PR fixes ZkConfigMigrationClient to do the sanitization correctly on both the read and write paths. We do de-sanitization before invoking the visitors, since after all it does not make sense to do the same de-sanitization step in each and every visitor. Additionally, this PR fixes a bug causing default entities to be converted incorrectly. For example, ClientQuotaEntity(user -> null) is stored under the /config/users/ znode in ZooKeeper. In KRaft it appears as a ClientQuotaRecord with EntityData(entityType=users, entityName=null). Prior to this PR, this was being converted to a ClientQuotaRecord with EntityData(entityType=users, entityName=""). That represents a quota on the user whose name is the empty string (yes, we allow users to name themselves with the empty string, sadly.) The confusion appears to have arisen because for TOPIC and BROKER configurations, the default ConfigResource is indeed the one named with the empty (not null) string. For example, the default topic configuration resource is ConfigResource(name="", type=TOPIC). However, things are different for client quotas. Default client quota entities in KRaft (and also in AdminClient) are represented by maps with null values. For example, the default User entity is represented by Map("user" -> null). In retrospect, using a map with null values was a poor choice; a Map> would have made more sense. However, this is the way the API currently is and we have to convert correctly. There was an additional level of confusion present in KAFKA-16222 where someone thought that using the ZooKeeper placeholder string "" in the AdminClient API would yield a default client quota entity. Thise seems to have been suggested by the ConfigEntityName class that was created recently. In fact, is not part of any public API in Kafka. Accordingly, this PR also renames ConfigEntityName.DEFAULT to ZooKeeperInternals.DEFAULT_STRING, to make it clear that the string is just a detail of the ZooKeeper implementation. It is not used in the Kafka API to indicate defaults. Hopefully this will avoid confusion in the future. Finally, the PR also creates KRaftClusterTest.testDefaultClientQuotas to get extra test coverage of setting default client quotas. Reviewers: Manikumar Reddy , Igor Soarez Conflicts: Do not backport the changes to create ZooKeeperInternals.DEFAULT_STRING to this branch, to make the cherry-pick smaller. --- .../main/scala/kafka/zk/ZkMigrationClient.scala| 4 - .../zk/migration/ZkConfigMigrationClient.scala | 89 -- .../kafka/server/KRaftClusterTest.scala| 63 +++ .../kafka/zk/ZkMigrationIntegrationTest.scala | 45 ++- .../zk/migration/ZkConfigMigrationClientTest.scala | 22 +++--- 5 files changed, 167 insertions(+), 56 deletions(-) diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index 76e0b47aee8..a11a84c017b 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.metadata._ import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.security.scram.ScramCredential -import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.common.{TopicIdPartition, Uuid} import org.apache.kafka.metadata.DelegationTokenData import org.apache.kafka.metadata.PartitionRegistration @@ -226,9 +225,6 @@ class ZkMigrationClient( entityDataList: util.List[ClientQuotaRecord.EntityData], quotas: util.Map[String, lang.Double] ): Unit = { -entityDataList.forEach(entityData => { - entityData.setEntityName(Sanitizer.desanitize(entityData.entityName())) -}) val batch = new util.ArrayList[
(kafka) branch 3.7 updated: KAFKA-16411: Correctly migrate default client quota entities (#15584)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new fedf8cfa7a2 KAFKA-16411: Correctly migrate default client quota entities (#15584) fedf8cfa7a2 is described below commit fedf8cfa7a2cc11dfb0c7c35e6eb9a9935a1e146 Author: Colin P. McCabe AuthorDate: Wed Mar 27 09:38:11 2024 -0700 KAFKA-16411: Correctly migrate default client quota entities (#15584) KAFKA-16222 fixed a bug whereby we didn't undo the name sanitization used on client quota entity names stored in ZooKeeper. However, it incorrectly claimed to fix the handling of default client quota entities. It also failed to correctly re-sanitize when syncronizing the data back to ZooKeeper. This PR fixes ZkConfigMigrationClient to do the sanitization correctly on both the read and write paths. We do de-sanitization before invoking the visitors, since after all it does not make sense to do the same de-sanitization step in each and every visitor. Additionally, this PR fixes a bug causing default entities to be converted incorrectly. For example, ClientQuotaEntity(user -> null) is stored under the /config/users/ znode in ZooKeeper. In KRaft it appears as a ClientQuotaRecord with EntityData(entityType=users, entityName=null). Prior to this PR, this was being converted to a ClientQuotaRecord with EntityData(entityType=users, entityName=""). That represents a quota on the user whose name is the empty string (yes, we allow users to name themselves with the empty string, sadly.) The confusion appears to have arisen because for TOPIC and BROKER configurations, the default ConfigResource is indeed the one named with the empty (not null) string. For example, the default topic configuration resource is ConfigResource(name="", type=TOPIC). However, things are different for client quotas. Default client quota entities in KRaft (and also in AdminClient) are represented by maps with null values. For example, the default User entity is represented by Map("user" -> null). In retrospect, using a map with null values was a poor choice; a Map> would have made more sense. However, this is the way the API currently is and we have to convert correctly. There was an additional level of confusion present in KAFKA-16222 where someone thought that using the ZooKeeper placeholder string "" in the AdminClient API would yield a default client quota entity. Thise seems to have been suggested by the ConfigEntityName class that was created recently. In fact, is not part of any public API in Kafka. Accordingly, this PR also renames ConfigEntityName.DEFAULT to ZooKeeperInternals.DEFAULT_STRING, to make it clear that the string is just a detail of the ZooKeeper implementation. It is not used in the Kafka API to indicate defaults. Hopefully this will avoid confusion in the future. Finally, the PR also creates KRaftClusterTest.testDefaultClientQuotas to get extra test coverage of setting default client quotas. Reviewers: Manikumar Reddy , Igor Soarez Conflicts: Do not backport the changes to create ZooKeeperInternals.DEFAULT_STRING to this branch, to make the cherry-pick smaller. --- .../main/scala/kafka/zk/ZkMigrationClient.scala| 4 - .../zk/migration/ZkConfigMigrationClient.scala | 89 -- .../kafka/server/KRaftClusterTest.scala| 63 +++ .../kafka/zk/ZkMigrationIntegrationTest.scala | 45 ++- .../zk/migration/ZkConfigMigrationClientTest.scala | 22 +++--- 5 files changed, 167 insertions(+), 56 deletions(-) diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index 76e0b47aee8..a11a84c017b 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.metadata._ import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.security.scram.ScramCredential -import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.common.{TopicIdPartition, Uuid} import org.apache.kafka.metadata.DelegationTokenData import org.apache.kafka.metadata.PartitionRegistration @@ -226,9 +225,6 @@ class ZkMigrationClient( entityDataList: util.List[ClientQuotaRecord.EntityData], quotas: util.Map[String, lang.Double] ): Unit = { -entityDataList.forEach(entityData => { - entityData.setEntityName(Sanitizer.desanitize(entityData.entityName())) -}) val batch = new util.ArrayList[
(kafka) branch trunk updated: KAFKA-16411: Correctly migrate default client quota entities (#15584)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 8d914b543d5 KAFKA-16411: Correctly migrate default client quota entities (#15584) 8d914b543d5 is described below commit 8d914b543d5d23031fe178d424f45789eaa8d1fc Author: Colin Patrick McCabe AuthorDate: Tue Mar 26 16:49:38 2024 -0700 KAFKA-16411: Correctly migrate default client quota entities (#15584) KAFKA-16222 fixed a bug whereby we didn't undo the name sanitization used on client quota entity names stored in ZooKeeper. However, it incorrectly claimed to fix the handling of default client quota entities. It also failed to correctly re-sanitize when syncronizing the data back to ZooKeeper. This PR fixes ZkConfigMigrationClient to do the sanitization correctly on both the read and write paths. We do de-sanitization before invoking the visitors, since after all it does not make sense to do the same de-sanitization step in each and every visitor. Additionally, this PR fixes a bug causing default entities to be converted incorrectly. For example, ClientQuotaEntity(user -> null) is stored under the /config/users/ znode in ZooKeeper. In KRaft it appears as a ClientQuotaRecord with EntityData(entityType=users, entityName=null). Prior to this PR, this was being converted to a ClientQuotaRecord with EntityData(entityType=users, entityName=""). That represents a quota on the user whose name is the empty string (yes, we allow users to name themselves with the empty string, sadly.) The confusion appears to have arisen because for TOPIC and BROKER configurations, the default ConfigResource is indeed the one named with the empty (not null) string. For example, the default topic configuration resource is ConfigResource(name="", type=TOPIC). However, things are different for client quotas. Default client quota entities in KRaft (and also in AdminClient) are represented by maps with null values. For example, the default User entity is represented by Map("user" -> null). In retrospect, using a map with null values was a poor choice; a Map> would have made more sense. However, this is the way the API currently is and we have to convert correctly. There was an additional level of confusion present in KAFKA-16222 where someone thought that using the ZooKeeper placeholder string "" in the AdminClient API would yield a default client quota entity. Thise seems to have been suggested by the ConfigEntityName class that was created recently. In fact, is not part of any public API in Kafka. Accordingly, this PR also renames ConfigEntityName.DEFAULT to ZooKeeperInternals.DEFAULT_STRING, to make it clear that the string is just a detail of the ZooKeeper implementation. It is not used in the Kafka API to indicate defaults. Hopefully this will avoid confusion in the future. Finally, the PR also creates KRaftClusterTest.testDefaultClientQuotas to get extra test coverage of setting default client quotas. Reviewers: Manikumar Reddy , Igor Soarez --- .../src/main/scala/kafka/admin/ConfigCommand.scala | 12 +-- .../scala/kafka/server/ClientQuotaManager.scala| 13 ++- .../main/scala/kafka/server/ConfigHandler.scala| 8 +- .../scala/kafka/server/DynamicBrokerConfig.scala | 4 +- .../main/scala/kafka/server/DynamicConfig.scala| 4 +- .../main/scala/kafka/server/ZkAdminManager.scala | 8 +- .../metadata/ClientQuotaMetadataManager.scala | 14 +-- .../server/metadata/DynamicConfigPublisher.scala | 4 +- .../kafka/server/metadata/ZkConfigRepository.scala | 4 +- core/src/main/scala/kafka/zk/AdminZkClient.scala | 8 +- .../main/scala/kafka/zk/ZkMigrationClient.scala| 4 - .../zk/migration/ZkConfigMigrationClient.scala | 99 +++--- .../kafka/admin/ConfigCommandIntegrationTest.scala | 4 +- .../kafka/server/KRaftClusterTest.scala| 65 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 46 +- .../scala/unit/kafka/admin/ConfigCommandTest.scala | 12 +-- .../unit/kafka/server/ClientQuotaManagerTest.scala | 42 - .../kafka/server/ClientQuotasRequestTest.scala | 5 +- .../kafka/server/DynamicConfigChangeTest.scala | 4 +- .../zk/migration/ZkConfigMigrationClientTest.scala | 22 ++--- ...nfigEntityName.java => ZooKeeperInternals.java} | 10 ++- 21 files changed, 251 insertions(+), 141 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index bcbeedc468e..d03d7dfb9b5 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafk
(kafka) branch KAFKA-16411 created (now bf1900e9971)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16411 in repository https://gitbox.apache.org/repos/asf/kafka.git at bf1900e9971 Improve tests This branch includes the following new commits: new cbe5ada027c Rename 'mangling' to 'sanitization' to be more consistent new bf1900e9971 Improve tests The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) 02/02: Improve tests
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16411 in repository https://gitbox.apache.org/repos/asf/kafka.git commit bf1900e9971cddd6c08e38baf40c9ae04dfc84cc Author: Colin P. McCabe AuthorDate: Mon Mar 25 17:01:37 2024 -0700 Improve tests --- .../kafka/zk/ZkMigrationIntegrationTest.scala | 22 +- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 16d68b3ae38..a3e5ecc0814 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.resource.ResourceType.TOPIC import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils -import org.apache.kafka.common.utils.SecurityUtils +import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils} import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.metadata.authorizer.StandardAcl import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState @@ -881,11 +881,14 @@ class ZkMigrationIntegrationTest { def alterClientQuotas(admin: Admin): AlterClientQuotasResult = { val quotas = new util.ArrayList[ClientQuotaAlteration]() quotas.add(new ClientQuotaAlteration( - new ClientQuotaEntity(Map("user" -> "user1").asJava), + new ClientQuotaEntity(Map("user" -> "user@1").asJava), List(new ClientQuotaAlteration.Op("consumer_byte_rate", 1000.0)).asJava)) quotas.add(new ClientQuotaAlteration( - new ClientQuotaEntity(Map("user" -> "user1", "client-id" -> "clientA").asJava), + new ClientQuotaEntity(Map("user" -> "user@1", "client-id" -> "clientA").asJava), List(new ClientQuotaAlteration.Op("consumer_byte_rate", 800.0), new ClientQuotaAlteration.Op("producer_byte_rate", 100.0)).asJava)) +quotas.add(new ClientQuotaAlteration( + new ClientQuotaEntity(Collections.singletonMap("user", null)), + List(new ClientQuotaAlteration.Op("consumer_byte_rate", 900.0), new ClientQuotaAlteration.Op("producer_byte_rate", 100.0)).asJava)) quotas.add(new ClientQuotaAlteration( new ClientQuotaEntity(Map("ip" -> "8.8.8.8").asJava), List(new ClientQuotaAlteration.Op("connection_creation_rate", 10.0)).asJava)) @@ -903,7 +906,7 @@ class ZkMigrationIntegrationTest { val alterations = new util.ArrayList[UserScramCredentialAlteration]() alterations.add(new UserScramCredentialUpsertion("user1", new ScramCredentialInfo(ScramMechanism.SCRAM_SHA_256, 8191), "password1")) -alterations.add(new UserScramCredentialUpsertion("user2", +alterations.add(new UserScramCredentialUpsertion("user@2", new ScramCredentialInfo(ScramMechanism.SCRAM_SHA_256, 8192), "password2")) admin.alterUserScramCredentials(alterations) } @@ -918,20 +921,21 @@ class ZkMigrationIntegrationTest { def verifyClientQuotas(zkClient: KafkaZkClient): Unit = { TestUtils.retry(1) { - assertEquals("1000", zkClient.getEntityConfigs(ConfigType.USER, "user1").getProperty("consumer_byte_rate")) - assertEquals("800", zkClient.getEntityConfigs("users/user1/clients", "clientA").getProperty("consumer_byte_rate")) - assertEquals("100", zkClient.getEntityConfigs("users/user1/clients", "clientA").getProperty("producer_byte_rate")) + assertEquals("1000", zkClient.getEntityConfigs(ConfigType.USER, Sanitizer.sanitize("user@1")).getProperty("consumer_byte_rate")) + assertEquals("900", zkClient.getEntityConfigs(ConfigType.USER, "").getProperty("consumer_byte_rate")) + assertEquals("800", zkClient.getEntityConfigs("users/" + Sanitizer.sanitize("user@1") + "/clients", "clientA").getProperty("consumer_byte_rate")) + assertEquals("100", zkClient.getEntityConfigs("users/" + Sanitizer.sanitize("user@1") + "/clients", "clientA").getProperty("producer_byte_rate")) assertEquals("10", zkClient.getEntityConfigs(ConfigType.IP, "8.8.8.8").getProperty("connection_creation_rate")) }
(kafka) 01/02: Rename 'mangling' to 'sanitization' to be more consistent
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16411 in repository https://gitbox.apache.org/repos/asf/kafka.git commit cbe5ada027c1882b74ea55e998325f62be2f611b Author: Colin P. McCabe AuthorDate: Mon Mar 25 12:23:08 2024 -0700 Rename 'mangling' to 'sanitization' to be more consistent --- .../kafka/zk/migration/ZkConfigMigrationClient.scala | 18 +- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala index 04a58894ae7..fda6e93c249 100644 --- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala @@ -21,7 +21,7 @@ import kafka.server.{DynamicBrokerConfig, DynamicConfig, ZkAdminManager} import kafka.utils.Logging import kafka.zk.ZkMigrationClient.{logAndRethrow, wrapZkException} import kafka.zk._ -import kafka.zk.migration.ZkConfigMigrationClient.getMangledClientQuotaZNodeName +import kafka.zk.migration.ZkConfigMigrationClient.getSanitizedClientQuotaZNodeName import kafka.zookeeper.{CreateRequest, DeleteRequest, SetDataRequest} import org.apache.kafka.clients.admin.ScramMechanism import org.apache.kafka.common.config.types.Password @@ -81,8 +81,8 @@ class ZkConfigMigrationClient( // which have their names set to the empty string instead. result.setEntityName(null) } else { - // ZNode names are mangled before being stored in ZooKeeper. - // For example, @ is turned into %40. Undo the mangling here. + // ZNode names are sanitized before being stored in ZooKeeper. + // For example, @ is turned into %40. Undo the sanitization here. result.setEntityName(Sanitizer.desanitize(znodeName)) } result @@ -261,9 +261,9 @@ class ZkConfigMigrationClient( scram: util.Map[String, String], state: ZkMigrationLeadershipState ): ZkMigrationLeadershipState = wrapZkException { -val user: Option[String] = getMangledClientQuotaZNodeName(entity, ClientQuotaEntity.USER) -val client: Option[String] = getMangledClientQuotaZNodeName(entity, ClientQuotaEntity.CLIENT_ID) -val ip: Option[String] = getMangledClientQuotaZNodeName(entity, ClientQuotaEntity.IP) +val user: Option[String] = getSanitizedClientQuotaZNodeName(entity, ClientQuotaEntity.USER) +val client: Option[String] = getSanitizedClientQuotaZNodeName(entity, ClientQuotaEntity.CLIENT_ID) +val ip: Option[String] = getSanitizedClientQuotaZNodeName(entity, ClientQuotaEntity.IP) val props = new Properties() val (configType, path, configKeys) = if (user.isDefined && client.isEmpty) { @@ -369,7 +369,7 @@ object ZkConfigMigrationClient { * @param component The component that we want a znode name for. * @returnSome(znodeName) if there is a znode path; None otherwise. */ - def getMangledClientQuotaZNodeName( + def getSanitizedClientQuotaZNodeName( entity: util.Map[String, String], component: String ): Option[String] = { @@ -385,10 +385,10 @@ object ZkConfigMigrationClient { // "not present." This is an unfortunate API that should be revisited at some point. Some(ZooKeeperInternals.DEFAULT_STRING) } else { -// We found a non-null value, and now we need to mangle it. For example, "c@@ldude" will +// We found a non-null value, and now we need to sanitize it. For example, "c@@ldude" will // turn into c%40%40ldude, so that we can use it as a znode name in ZooKeeper. Some(Sanitizer.sanitize(rawValue)) } } } -} \ No newline at end of file +}
(kafka) branch cmccabe_2024_03_22_fix_closeables updated (bf79133bf02 -> ad3876dfdfe)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch cmccabe_2024_03_22_fix_closeables in repository https://gitbox.apache.org/repos/asf/kafka.git discard bf79133bf02 MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest add ad3876dfdfe MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest This update added new revisions after undoing existing revisions. That is to say, some revisions that were in the old version of the branch are not in the new version. This situation occurs when a user --force pushes a change and generates a repository containing something like this: * -- * -- B -- O -- O -- O (bf79133bf02) \ N -- N -- N refs/heads/cmccabe_2024_03_22_fix_closeables (ad3876dfdfe) You should already have received notification emails for all of the O revisions, and so the following emails describe only the N revisions from the common base, B. Any revisions marked "omit" are not gone; other references still refer to them. Any revisions marked "discard" are gone forever. No new revisions were added by this update. Summary of changes: .../java/org/apache/kafka/controller/QuorumControllerTest.java | 10 +- 1 file changed, 5 insertions(+), 5 deletions(-)
(kafka) 01/01: MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch cmccabe_2024_03_22_fix_closeables in repository https://gitbox.apache.org/repos/asf/kafka.git commit bf79133bf02c7e6425ac8ea61cf3f13a7d3e39ca Author: Colin P. McCabe AuthorDate: Fri Mar 22 13:18:59 2024 -0700 MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest --- .../kafka/controller/QuorumControllerTest.java | 37 ++ 1 file changed, 17 insertions(+), 20 deletions(-) diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index bcdb3ed7919..462cd726857 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -1540,15 +1540,14 @@ public class QuorumControllerTest { @Test public void testMigrationsEnabledForOldBootstrapMetadataVersion() throws Exception { try ( -LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build() +LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build(); +QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). +setControllerBuilderInitializer(controllerBuilder -> { +controllerBuilder.setZkMigrationEnabled(true); +}). + setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV0, "test")). +build(); ) { -QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). -setControllerBuilderInitializer(controllerBuilder -> { -controllerBuilder.setZkMigrationEnabled(true); -}). - setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV0, "test")); - -QuorumControllerTestEnv controlEnv = controlEnvBuilder.build(); QuorumController active = controlEnv.activeController(); assertEquals(ZkMigrationState.NONE, active.appendReadEvent("read migration state", OptionalLong.empty(), () -> active.featureControl().zkMigrationState()).get(30, TimeUnit.SECONDS)); @@ -1658,12 +1657,12 @@ public class QuorumControllerTest { @Test public void testFailoverDuringMigrationTransaction() throws Exception { try ( -LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).build() -) { -QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). +LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).build(); +QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). setControllerBuilderInitializer(controllerBuilder -> controllerBuilder.setZkMigrationEnabled(true)). - setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test")); -QuorumControllerTestEnv controlEnv = controlEnvBuilder.build(); + setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test")). +build(); +) { QuorumController active = controlEnv.activeController(true); ZkRecordConsumer migrationConsumer = active.zkRecordConsumer(); migrationConsumer.beginMigration().get(30, TimeUnit.SECONDS); @@ -1703,18 +1702,17 @@ public class QuorumControllerTest { @EnumSource(value = MetadataVersion.class, names = {"IBP_3_4_IV0", "IBP_3_5_IV0", "IBP_3_6_IV0", "IBP_3_6_IV1"}) public void testBrokerHeartbeatDuringMigration(MetadataVersion metadataVersion) throws Exception { try ( -LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build() -) { -QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). +LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build(); +QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). setControllerBuilderInitializer(controllerBuilder -> controllerBuilder .setZkMigrationEnabled(true) .setMaxIdleIntervalNs(OptionalLong.of(TimeUnit.MILLISECONDS.toNanos(100))) ). - setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test")); -QuorumControllerTestEnv controlEnv = controlEnvBuilder.build(); + setBootstrapMetadata(BootstrapMetadata
(kafka) branch cmccabe_2024_03_22_fix_closeables created (now bf79133bf02)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch cmccabe_2024_03_22_fix_closeables in repository https://gitbox.apache.org/repos/asf/kafka.git at bf79133bf02 MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest This branch includes the following new commits: new bf79133bf02 MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) branch 3.6 updated: KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new 4cec48f86ec KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293) 4cec48f86ec is described below commit 4cec48f86ec95848b2e3893f04dc90d1d8415a47 Author: Colin Patrick McCabe AuthorDate: Fri Feb 2 15:49:10 2024 -0800 KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293) While migrating from ZK mode to KRaft mode, the broker passes through a "hybrid" phase, in which it receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft controller. For the most part, these RPCs can be handled just like their traditional equivalents from a ZK-based controller. However, there is one thing that is different: the way topic deletions are handled. In ZK mode, there is a "deleting" state which topics enter prior to being completely removed. Partitions stay in this state until they are removed from the disks of all replicas. And partitions associated with these deleting topics show up in the UMR and LAIR as having a leader of -2 (which is not a valid broker ID, of course, because it's negative). When brokers receive these RPCs, they know to remove the associated partitions from their metadata caches, and disks. When a full UMR or ISR is sent, deleting partitions are included as well. In hybrid mode, in contrast, there is no "deleting" state. Topic deletion happens immediately. We can do this because we know that we have topic IDs that are never reused. This means that we can always tell the difference between a broker that had an old version of some topic, and a broker that has a new version that was re-created with the same name. To make this work, when handling a full UMR or LAIR, hybrid brokers must compare the full state that was sent over the wire to their own local state, and adjust accordingly. Prior to this PR, the code for handling those adjustments had several major flaws. The biggest flaw is that it did not correctly handle the "re-creation" case where a topic named FOO appears in the RPC, but with a different ID than the broker's local FOO. Another flaw is that a problem with a single partition would prevent handling the whole request. In ZkMetadataCache.scala, we handle full UMR requests from KRaft controllers by rewriting the UMR so that it contains the implied deletions. I fixed this code so that deletions always appear at the start of the list of topic states. This is important for the re-creation case since it means that a single request can both delete the old FOO and add a new FOO to the cache. Also, rather than modifying the requesst in-place, as the previous code did, I build a whole new request with the desired list of topic states. This is much safer because it avoids unforseen interactions with other parts of the code that deal with requests (like request logging). While this new copy may sound expensive, it should actually not be. We are doing a "shallow copy" which references the previous list topic state entries. I also reworked ZkMetadataCache.updateMetadata so that if a partition is re-created, it does not appear in the returned set of deleted TopicPartitions. Since this set is used only by the group manager, this seemed appropriate. (If I was in the consumer group for the previous iteration of FOO, I should still be in the consumer group for the new iteration.) On the ReplicaManager.scala side, we handle full LAIR requests by treating anything which does not appear in them as a "stray replica." (But we do not rewrite the request objects as we do with UMR.) I moved the logic for finding stray replicas from ReplicaManager into LogManager. It makes more sense there, since the information about what is on-disk is managed in LogManager. Also, the stray replica detection logic for KRaft mode is there, so it makes sense to put the stray replica detection logic for hybrid mode there as well. Since the stray replica detection is now in LogManager, I moved the unit tests there as well. Previously some of those tests had been in BrokerMetadataPublisherTest for historical reasons. The main advantage of the new LAIR logic is that it takes topic ID into account. A replica can be a stray even if the LAIR contains a topic of the given name, but a different ID. I also moved the stray replica handling earlier in the becomeLeaderOrFollower function, so that we could correctly handle the "delete and re-create FOO" case. Reviewers: David Arthur Conflicts: For this cherry-pic
(kafka) branch 3.6 updated: HOTFIX for KAFKA-16126 test in 3.6
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new 05a36eb10bf HOTFIX for KAFKA-16126 test in 3.6 05a36eb10bf is described below commit 05a36eb10bf23a7515066926449b057e970d0428 Author: Colin P. McCabe AuthorDate: Tue Mar 12 16:59:41 2024 -0700 HOTFIX for KAFKA-16126 test in 3.6 --- core/src/test/java/kafka/testkit/KafkaClusterTestKit.java | 4 +--- core/src/test/java/kafka/testkit/TestKitNodes.java| 4 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index e09ee49402f..f6e4566b479 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -221,8 +221,6 @@ public class KafkaClusterTestKit implements AutoCloseable { ThreadUtils.createThreadFactory("kafka-cluster-test-kit-executor-%d", false)); for (ControllerNode node : nodes.controllerNodes().values()) { setupNodeDirectories(baseDirectory, node.metadataDirectory(), Collections.emptyList()); -BootstrapMetadata bootstrapMetadata = BootstrapMetadata. -fromVersion(nodes.bootstrapMetadataVersion(), "testkit"); SharedServer sharedServer = new SharedServer(createNodeConfig(node), MetaProperties.apply(nodes.clusterId().toString(), node.id()), Time.SYSTEM, @@ -234,7 +232,7 @@ public class KafkaClusterTestKit implements AutoCloseable { controller = new ControllerServer( sharedServer, KafkaRaftServer.configSchema(), -bootstrapMetadata); +nodes.bootstrapMetadata()); } catch (Throwable e) { log.error("Error creating controller {}", node.id(), e); Utils.swallow(log, Level.WARN, "sharedServer.stopForController error", () -> sharedServer.stopForController()); diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java index da4e10a4431..90d095f70c7 100644 --- a/core/src/test/java/kafka/testkit/TestKitNodes.java +++ b/core/src/test/java/kafka/testkit/TestKitNodes.java @@ -165,6 +165,10 @@ public class TestKitNodes { return bootstrapMetadata.metadataVersion(); } +public BootstrapMetadata bootstrapMetadata() { +return bootstrapMetadata; +} + public Map controllerNodes() { return controllerNodes; }
(kafka) branch 3.6 updated: KAFKA-16171: Fix ZK migration controller race #15238
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new 5d3e691e47b KAFKA-16171: Fix ZK migration controller race #15238 5d3e691e47b is described below commit 5d3e691e47ba35b1cac481bcf158890413f19e3c Author: David Arthur AuthorDate: Fri Jan 19 13:38:37 2024 -0500 KAFKA-16171: Fix ZK migration controller race #15238 This patch causes the active KRaftMigrationDriver to reload the /migration ZK state after electing itself as the leader in ZK. This closes a race condition where the previous active controller could make an update to /migration after the new leader was elected. The update race was not actually a problem regarding the data since both controllers would be syncing the same state from KRaft to ZK, but the change to the znode causes the new controller to fail on the zk version check on /migration. This patch also fixes a as-yet-unseen bug where the active controllers failing to elect itself via claimControllerLeadership would not retry. Reviewers: Colin P. McCabe --- .../kafka/zk/ZkMigrationFailoverTest.scala | 284 + .../metadata/migration/KRaftMigrationDriver.java | 14 +- .../migration/ZkMigrationLeadershipState.java | 9 +- .../migration/CapturingMigrationClient.java| 12 +- 4 files changed, 312 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala new file mode 100644 index 000..1b297f4fc9b --- /dev/null +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.zk + +import kafka.utils.{Logging, PasswordEncoder, TestUtils} +import org.apache.kafka.clients.ApiVersions +import org.apache.kafka.common.{Node, Uuid} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, TopicRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.controller.QuorumFeatures +import org.apache.kafka.controller.metrics.QuorumControllerMetrics +import org.apache.kafka.image.loader.LogDeltaManifest +import org.apache.kafka.image.publisher.MetadataPublisher +import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} +import org.apache.kafka.metadata.KafkaConfigSchema +import org.apache.kafka.metadata.migration._ +import org.apache.kafka.raft.{LeaderAndEpoch, OffsetAndEpoch} +import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.FaultHandler +import org.apache.zookeeper.client.ZKClientConfig +import org.junit.jupiter.api.Assertions.{assertTrue, fail} +import org.junit.jupiter.api.Test + +import java.util +import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.{Optional, OptionalInt} +import scala.collection.mutable + +class ZkMigrationFailoverTest extends Logging { + + class CapturingFaultHandler(nodeId: Int) extends FaultHandler { +val faults = mutable.Buffer[Throwable]() +var future: CompletableFuture[Throwable] = CompletableFuture.completedFuture(new RuntimeException()) +var waitingForMsg = "" + +override def handleFault(failureMessage: String, cause: Throwable): RuntimeException = { + error(s"Fault handled on node $nodeId", cause) + faults.append(cause) + if (!future.isDone && cause.getMessage.contains(waitingForMsg)) { +future.complete(cause) + } + new RuntimeException(cause) +} + +def checkAndClear(verifier: (Seq[Throwable]) => Unit): Unit = { + val faultsSoFar = faults.toSeq + try { +verifier.apply(faultsSoFar) + } catch { +case ae: AssertionError => fail(s"Assertion failed. Faults on $nodeId were: $faultsSoFar", ae) + } +} + +def waitForError(message: String): CompletableFuture[Throwable] = { + future = n
(kafka) branch 3.6 updated: KAFKA-16216: Reduce batch size for initial metadata load during ZK migration
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new f734f32b85f KAFKA-16216: Reduce batch size for initial metadata load during ZK migration f734f32b85f is described below commit f734f32b85f3846bf6ca017483b31d99a2ed5b72 Author: David Arthur AuthorDate: Thu Feb 1 15:29:07 2024 -0800 KAFKA-16216: Reduce batch size for initial metadata load during ZK migration During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but 200 seems better. This PR also adds an internal configuration to control this batch size, for testing purposes. Reviewers: Colin P. McCabe --- .../main/scala/kafka/server/ControllerServer.scala | 1 + core/src/main/scala/kafka/server/KafkaConfig.scala | 5 + .../metadata/migration/KRaftMigrationDriver.java | 17 +++--- .../migration/KRaftMigrationDriverTest.java| 26 +- 4 files changed, 36 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 5ffc382cf8d..4ba7f10d2a0 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -287,6 +287,7 @@ class ControllerServer( .setQuorumFeatures(quorumFeatures) .setConfigSchema(configSchema) .setControllerMetrics(quorumControllerMetrics) + .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize) .setTime(time) .build() migrationDriver.start() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 618faeee790..4a11c70c6a5 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -91,6 +91,7 @@ object Defaults { /** KRaft mode configs */ val EmptyNodeId: Int = -1 val ServerMaxStartupTimeMs = Long.MaxValue + val MigrationMetadataMinBatchSize: Int = 200 /* Authorizer Configuration ***/ val AuthorizerClassName = "" @@ -398,6 +399,7 @@ object KafkaConfig { /** ZK to KRaft Migration configs */ val MigrationEnabledProp = "zookeeper.metadata.migration.enable" + val MigrationMetadataMinBatchSizeProp = "zookeeper.metadata.migration.min.batch.size" /* Authorizer Configuration ***/ val AuthorizerClassNameProp = "authorizer.class.name" @@ -1236,6 +1238,8 @@ object KafkaConfig { .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.MetadataMaxIdleIntervalMs, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc) .defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.ServerMaxStartupTimeMs, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") + .defineInternal(MigrationMetadataMinBatchSizeProp, INT, Defaults.MigrationMetadataMinBatchSize, atLeast(1), +MEDIUM, "Soft minimum batch size to use when migrating metadata from ZooKeeper to KRaft") /* Authorizer Configuration ***/ .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) @@ -1735,6 +1739,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp) + val migrationMetadataMinBatchSize: Int = getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp) private def parseProcessRoles(): Set[ProcessRole] = { val roles = getList(KafkaConfig.ProcessRolesProp).asScala.map { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 52a9d74b23f..172e5f2f601 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -90,8 +90,6 @@ public class KRaftMigrationDriver implements MetadataPublisher { */ final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000; -final static int MIGRATION_MIN_BATCH_SIZE = 1_000; - private final Time time; private final Logger log; private final int nodeId; @@ -110,6 +108,7 @@ public class KRaftMigrationDriver implements MetadataPublisher { * MetadataPublish
(kafka) branch cmccabe_2024_02_26_fix deleted (was fc13cd04d7f)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch cmccabe_2024_02_26_fix in repository https://gitbox.apache.org/repos/asf/kafka.git was fc13cd04d7f MINOR: remove test constructor for PartitionAssignment The revisions that were on this branch are still contained in other references; therefore, this change does not discard any commits from the repository.
(kafka) branch trunk updated: MINOR: remove test constructor for PartitionAssignment (#15435)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 2c1943d836d MINOR: remove test constructor for PartitionAssignment (#15435) 2c1943d836d is described below commit 2c1943d836d828755af382b9d7996ff092854fe2 Author: Colin Patrick McCabe AuthorDate: Tue Mar 5 12:02:19 2024 -0800 MINOR: remove test constructor for PartitionAssignment (#15435) Remove the test constructor for PartitionAssignment and remove the TODO. Also add KRaftClusterTest.testCreatePartitions to get more coverage for createPartitions. Reviewers: David Arthur , Chia-Ping Tsai --- .../kafka/server/KRaftClusterTest.scala| 35 +- .../metadata/placement/PartitionAssignment.java| 6 .../controller/PartitionChangeBuilderTest.java | 6 ++-- .../PartitionReassignmentReplicasTest.java | 22 +++--- .../controller/ReplicationControlManagerTest.java | 18 +-- .../placement/PartitionAssignmentTest.java | 12 +--- .../placement/StripedReplicaPlacerTest.java| 19 ++-- .../metadata/placement/TopicAssignmentTest.java| 9 +++--- 8 files changed, 80 insertions(+), 47 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 130d0e5642e..3be1b400ab5 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter} import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.ConfigResource.Type -import org.apache.kafka.common.errors.{PolicyViolationException, UnsupportedVersionException} +import org.apache.kafka.common.errors.{InvalidPartitionsException,PolicyViolationException, UnsupportedVersionException} import org.apache.kafka.common.message.DescribeClusterRequestData import org.apache.kafka.common.metadata.{ConfigRecord, FeatureLevelRecord} import org.apache.kafka.common.metrics.Metrics @@ -792,6 +792,39 @@ class KRaftClusterTest { } } + @ParameterizedTest + @ValueSource(strings = Array("3.7-IV0", "3.7-IV2")) + def testCreatePartitions(metadataVersionString: String): Unit = { +val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). +setNumBrokerNodes(4). + setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)). +setNumControllerNodes(3).build()). + build() +try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + val admin = Admin.create(cluster.clientProperties()) + try { +val createResults = admin.createTopics(Arrays.asList( + new NewTopic("foo", 1, 3.toShort), + new NewTopic("bar", 2, 3.toShort))).values() +createResults.get("foo").get() +createResults.get("bar").get() +val increaseResults = admin.createPartitions(Map( + "foo" -> NewPartitions.increaseTo(3), + "bar" -> NewPartitions.increaseTo(2)).asJava).values() +increaseResults.get("foo").get() +assertEquals(classOf[InvalidPartitionsException], assertThrows( + classOf[ExecutionException], () => increaseResults.get("bar").get()).getCause.getClass) + } finally { +admin.close() + } +} finally { + cluster.close() +} + } private def clusterImage( cluster: KafkaClusterTestKit, brokerId: Int diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java index 177d5311afd..a7012d1505c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java @@ -17,7 +17,6 @@ package org.apache.kafka.metadata.placement; -import org.apache.kafka.common.DirectoryId; import org.apache.kafka.common.Uuid; import java.util.ArrayList; @@ -39,11 +38,6 @@ public class PartitionAssignment { private final List replicas; private final List directories; -// TODO remove -- just here for testing -public PartitionAssignment(List replicas) { -this(replicas, brokerId -> DirectoryId.UNASSIGNED); -} - public PartitionAssignment(List replicas, DefaultDirProvider defaultDirProvider) { this.replicas = Collections.unmodifiableList(new
(kafka) branch cmccabe_2024_02_26_fix updated (925c61a99ce -> fc13cd04d7f)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch cmccabe_2024_02_26_fix in repository https://gitbox.apache.org/repos/asf/kafka.git discard 925c61a99ce MINOR: remove test constructor for PartitionAssignment add 027fad4b2a0 KAFKA-16277: AbstractStickyAssignor - Sort owned TopicPartitions by partition when reassigning (#15416) add ddfcc333f8d KAFKA-16226 Add test for concurrently updatingMetadata and fetching snapshot/cluster (#15385) add 1c9f360f4af KAFKA-15215: migrate StreamedJoinTest to Mockito (#15424) add 3075a3f144c MINOR: Add 3.7.0 to system tests (#15436) add 09793275208 KAFKA-16306: fix GroupCoordinatorService logger (#15433) add 5d6936a4992 KAFKA-16305: Avoid optimisation in handshakeUnwrap (#15434) add 53c41aca7ba KAFKA-16116: Rebalance Metrics for AsyncKafkaConsumer (#15339) add 52289c92be4 MINOR: Optimize EventAccumulator (#15430) add 1bb9a851744 MINOR: Remove the space between two words (#15439) add 55a6d30ccbe KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#15213) add 96c68096a26 KAFKA-15462: Add Group Type Filter for List Group to the Admin Client (#15150) add f8eb4294d67 KAFKA-16191: Clean up of consumer client internal events (#15438) add d066b94c810 MINOR: Fix UpdatedImage and HighWatermarkUpdated events' logs (#15432) add c8843f06841 KAFKA-16167: Disable wakeups during autocommit on close (#15445) add 8e1516f88b8 KAFKA-16261: updateSubscription fails if already empty subscription (#15440) add 52a3fa07446 KAFKA-15878: KIP-768 - Extend support for opaque (i.e. non-JWT) tokens in SASL/OAUTHBEARER (#14818) add 21a5bbd84c3 MINOR: Upgrade jqwik to version 1.8.3 (#14365) add 44af72fd773 MINOR: fix link for ListTransactionsOptions#filterOnDuration (#15459) add 907e945c0b7 MINOR: fix SessionStore java doc (#15412) add 2c0cab39aed MINOR: Remove unnecessary easymock/powermock dependencies (#15460) add 7dbdc15c668 KAFKA-15625: Do not flush global state store at each commit (#15361) add 21a888c4ca0 MINOR: Updating comments to match the code (#15388) add aa0443eb607 KAFKA-16285: Make group metadata available when a new assignment is set (#15426) add c254b22a487 MINOR: simplify ensure topic exists condition (#15458) add 4f92a3f0afd KAFKA-14747: record discarded FK join subscription responses (#15395) add 99e511c706b KAFKA-16288, KAFKA-16289: Fix Values convertToDecimal exception and parseString corruption (#15399) add 47792770a28 KAFKA-16169: FencedException in commitAsync not propagated without callback (#15437) add eea369af947 KAFKA-14588 Log cleaner configuration move to CleanerConfig (#15387) add 2f401ff4c85 MINOR: parameterize group-id in GroupMetadataManagerTestContext (#15467) add fc13cd04d7f MINOR: remove test constructor for PartitionAssignment This update added new revisions after undoing existing revisions. That is to say, some revisions that were in the old version of the branch are not in the new version. This situation occurs when a user --force pushes a change and generates a repository containing something like this: * -- * -- B -- O -- O -- O (925c61a99ce) \ N -- N -- N refs/heads/cmccabe_2024_02_26_fix (fc13cd04d7f) You should already have received notification emails for all of the O revisions, and so the following emails describe only the N revisions from the common base, B. Any revisions marked "omit" are not gone; other references still refer to them. Any revisions marked "discard" are gone forever. No new revisions were added by this update. Summary of changes: build.gradle | 11 +- checkstyle/import-control-core.xml | 1 + checkstyle/import-control.xml | 2 + checkstyle/suppressions.xml| 1 + .../java/org/apache/kafka/clients/admin/Admin.java | 2 +- .../kafka/clients/admin/ConsumerGroupListing.java | 64 ++- .../kafka/clients/admin/KafkaAdminClient.java | 20 +- .../clients/admin/ListConsumerGroupsOptions.java | 25 +- .../consumer/internals/AbstractStickyAssignor.java | 1 + .../consumer/internals/AsyncKafkaConsumer.java | 230 +- .../consumer/internals/CommitRequestManager.java | 2 +- .../internals/CoordinatorRequestManager.java | 6 +- .../internals/HeartbeatRequestManager.java | 25 +- .../consumer/internals/MembershipManager.java | 7 +- .../consumer/internals/MembershipManagerImpl.java | 109 +++-- .../internals/OffsetCommitCallbackInvoker.java | 10 - .../consumer/internals/OffsetsRequestManager.java | 4 +- .../consumer/internals/RequestManagers.java| 7 +- .../clients/consumer/internals/RequestState.java | 2 +- .../clients/consumer/internals/WakeupTrigger.java |
(kafka) 01/01: MINOR: remove test constructor for PartitionAssignment
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch cmccabe_2024_02_26_fix in repository https://gitbox.apache.org/repos/asf/kafka.git commit 925c61a99ce51103af1ab3188997def5ad1f767b Author: Colin P. McCabe AuthorDate: Mon Feb 26 13:24:16 2024 -0800 MINOR: remove test constructor for PartitionAssignment Remove the test constructor for PartitionAssignment and remove the TODO. --- .../kafka/server/KRaftClusterTest.scala| 39 +- .../metadata/placement/PartitionAssignment.java| 6 .../controller/PartitionChangeBuilderTest.java | 6 ++-- .../PartitionReassignmentReplicasTest.java | 22 ++-- .../controller/ReplicationControlManagerTest.java | 18 +- .../placement/PartitionAssignmentTest.java | 12 --- .../placement/StripedReplicaPlacerTest.java| 17 +- .../metadata/placement/TopicAssignmentTest.java| 9 ++--- 8 files changed, 83 insertions(+), 46 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 130d0e5642e..379a7dd24aa 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter} import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.ConfigResource.Type -import org.apache.kafka.common.errors.{PolicyViolationException, UnsupportedVersionException} +import org.apache.kafka.common.errors.{InvalidPartitionsException,PolicyViolationException, UnsupportedVersionException} import org.apache.kafka.common.message.DescribeClusterRequestData import org.apache.kafka.common.metadata.{ConfigRecord, FeatureLevelRecord} import org.apache.kafka.common.metrics.Metrics @@ -792,6 +792,43 @@ class KRaftClusterTest { } } + /** + * Test that setting the Confluent-specific configuration + * confluent.apply.create.topic.policy.to.create.partitions has the expected effect. + */ + @ParameterizedTest + @ValueSource(strings = Array("3.7-IV0", "3.7-IV2")) + def testCreatePartitions(metadataVersionString: String): Unit = { +val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). +setNumBrokerNodes(4). + setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)). +setNumControllerNodes(3).build()). + build() +try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + val admin = Admin.create(cluster.clientProperties()) + try { +val createResults = admin.createTopics(Arrays.asList( + new NewTopic("foo", 1, 3.toShort), + new NewTopic("bar", 2, 3.toShort))).values() +createResults.get("foo").get() +createResults.get("bar").get() +val increaseResults = admin.createPartitions(Map( + "foo" -> NewPartitions.increaseTo(3), + "bar" -> NewPartitions.increaseTo(2)).asJava).values() +increaseResults.get("foo").get() +assertEquals(classOf[InvalidPartitionsException], assertThrows( + classOf[ExecutionException], () => increaseResults.get("bar").get()).getCause.getClass) + } finally { +admin.close() + } +} finally { + cluster.close() +} + } private def clusterImage( cluster: KafkaClusterTestKit, brokerId: Int diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java index 177d5311afd..a7012d1505c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java @@ -17,7 +17,6 @@ package org.apache.kafka.metadata.placement; -import org.apache.kafka.common.DirectoryId; import org.apache.kafka.common.Uuid; import java.util.ArrayList; @@ -39,11 +38,6 @@ public class PartitionAssignment { private final List replicas; private final List directories; -// TODO remove -- just here for testing -public PartitionAssignment(List replicas) { -this(replicas, brokerId -> DirectoryId.UNASSIGNED); -} - public PartitionAssignment(List replicas, DefaultDirProvider defaultDirProvider) { this.replicas = Collections.unmodifiableList(new ArrayList<>(replicas)); Uuid[] directories = new Uuid[replicas.size()]; diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBui
(kafka) branch cmccabe_2024_02_26_fix created (now 925c61a99ce)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch cmccabe_2024_02_26_fix in repository https://gitbox.apache.org/repos/asf/kafka.git at 925c61a99ce MINOR: remove test constructor for PartitionAssignment This branch includes the following new commits: new 925c61a99ce MINOR: remove test constructor for PartitionAssignment The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) 01/01: MINOR: update leaderAndEpoch before initializing metadata publishers
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch init_metadata_publishers in repository https://gitbox.apache.org/repos/asf/kafka.git commit b31840f8d7bd6a88da4a7a1f190b0ab438b817ef Author: Colin P. McCabe AuthorDate: Tue Feb 13 16:47:43 2024 -0800 MINOR: update leaderAndEpoch before initializing metadata publishers --- .../src/main/scala/kafka/server/SharedServer.scala | 1 + .../apache/kafka/image/loader/MetadataLoader.java | 23 ++ .../kafka/image/loader/MetadataLoaderTest.java | 10 ++ 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 31b4957e3f5..4dac53ed292 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -285,6 +285,7 @@ class SharedServer( setThreadNamePrefix(s"kafka-${sharedServerConfig.nodeId}-"). setFaultHandler(metadataLoaderFaultHandler). setHighWaterMarkAccessor(() => _raftManager.client.highWatermark()). + setLeaderAndEpochAccessor(() => _raftManager.client.leaderAndEpoch()). setMetrics(metadataLoaderMetrics) loader = loaderBuilder.build() snapshotEmitter = new SnapshotEmitter.Builder(). diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java index c6194f4e6dd..e967228a59e 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java @@ -75,6 +75,7 @@ public class MetadataLoader implements RaftClient.Listener private FaultHandler faultHandler = (m, e) -> new FaultHandlerException(m, e); private MetadataLoaderMetrics metrics = null; private Supplier highWaterMarkAccessor = null; +private Supplier leaderAndEpochAccessor = () -> LeaderAndEpoch.UNKNOWN; public Builder setNodeId(int nodeId) { this.nodeId = nodeId; @@ -101,6 +102,11 @@ public class MetadataLoader implements RaftClient.Listener return this; } +public Builder setLeaderAndEpochAccessor(Supplier leaderAndEpochAccessor) { +this.leaderAndEpochAccessor = leaderAndEpochAccessor; +return this; +} + public Builder setMetrics(MetadataLoaderMetrics metrics) { this.metrics = metrics; return this; @@ -126,7 +132,8 @@ public class MetadataLoader implements RaftClient.Listener threadNamePrefix, faultHandler, metrics, -highWaterMarkAccessor); +highWaterMarkAccessor, +leaderAndEpochAccessor); } } @@ -157,6 +164,11 @@ public class MetadataLoader implements RaftClient.Listener */ private final Supplier highWaterMarkAccessor; +/** + * A function which supplies the current leader and epoch. + */ +private final Supplier leaderAndEpochAccessor; + /** * Publishers which haven't received any metadata yet. */ @@ -197,13 +209,15 @@ public class MetadataLoader implements RaftClient.Listener String threadNamePrefix, FaultHandler faultHandler, MetadataLoaderMetrics metrics, -Supplier highWaterMarkAccessor +Supplier highWaterMarkAccessor, +Supplier leaderAndEpochAccessor ) { this.log = logContext.logger(MetadataLoader.class); this.time = time; this.faultHandler = faultHandler; this.metrics = metrics; this.highWaterMarkAccessor = highWaterMarkAccessor; +this.leaderAndEpochAccessor = leaderAndEpochAccessor; this.uninitializedPublishers = new LinkedHashMap<>(); this.publishers = new LinkedHashMap<>(); this.image = MetadataImage.EMPTY; @@ -245,8 +259,9 @@ public class MetadataLoader implements RaftClient.Listener offset + " and high water mark is {}", where, highWaterMark.getAsLong()); return true; } -log.info("{}: The loader finished catching up to the current high water mark of {}", -where, highWaterMark.getAsLong()); +currentLeaderAndEpoch = leaderAndEpochAccessor.get(); +log.info("{}: The loader finished catching up to the current high water mark of {}, {}", +where, highWaterMark.getAsLong(), currentLeaderAndEpoch); catchingUp = false; return false; } diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java index 921c241a09a..ad771c7
(kafka) branch init_metadata_publishers created (now b31840f8d7b)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch init_metadata_publishers in repository https://gitbox.apache.org/repos/asf/kafka.git at b31840f8d7b MINOR: update leaderAndEpoch before initializing metadata publishers This branch includes the following new commits: new b31840f8d7b MINOR: update leaderAndEpoch before initializing metadata publishers The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) branch 3.5 updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.5 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.5 by this push: new 5e4ca35f539 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) 5e4ca35f539 is described below commit 5e4ca35f5393d17ae9a845892f8d3bc78c76a40f Author: David Arthur AuthorDate: Wed Feb 7 15:54:59 2024 -0500 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) Reviewers: Colin P. McCabe Conflicts: skip adding the error logger callback in KRaftMigrationZkWriter.handleAclsDelta to simplify the cherry-pick and avoid conflicts. --- .../zk/migration/ZkAclMigrationClientTest.scala| 150 - .../java/org/apache/kafka/image/AclsDelta.java | 13 -- .../metadata/migration/KRaftMigrationZkWriter.java | 62 - 3 files changed, 180 insertions(+), 45 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index 0107b9fa954..57960e26687 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -21,7 +21,7 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString import kafka.utils.TestUtils import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ -import org.apache.kafka.common.metadata.AccessControlEntryRecord +import org.apache.kafka.common.metadata.{AccessControlEntryRecord, RemoveAccessControlEntryRecord} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils @@ -189,4 +189,152 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { AclPermissionType.fromCode(acl1Resource3.permissionType())), resource3AclsInZk.head.ace) } + + def user(user: String): String = { +new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString + } + + def acl(resourceName: String, + resourceType: ResourceType, + resourcePattern: PatternType, + principal: String, + host: String = "*", + operation: AclOperation = AclOperation.READ, + permissionType: AclPermissionType = AclPermissionType.ALLOW + ): AccessControlEntryRecord = { +new AccessControlEntryRecord() + .setId(Uuid.randomUuid()) + .setHost(host) + .setOperation(operation.code()) + .setPrincipal(principal) + .setPermissionType(permissionType.code()) + .setPatternType(resourcePattern.code()) + .setResourceName(resourceName) + .setResourceType(resourceType.code()) + } + + @Test + def testDeleteOneAclOfMany(): Unit = { +zkClient.createAclPaths() +val topicName = "topic-" + Uuid.randomUuid() +val resource = new ResourcePattern(ResourceType.TOPIC, topicName, PatternType.LITERAL) + +// Create a delta with some ACLs +val delta = new MetadataDelta(MetadataImage.EMPTY) +val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("alice")) +val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("bob")) +val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("carol")) +delta.replay(acl1) +delta.replay(acl2) +delta.replay(acl3) +val image = delta.apply(MetadataProvenance.EMPTY) + +// Sync image to ZK +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient) +kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { + migrationState = operation.apply(migrationState) +}) + +// verify 3 ACLs in ZK +val aclsInZk = zkClient.getVersionedAclsForResource(resource).acls +assertEquals(3, aclsInZk.size) + +// Delete one of the ACLs +val delta2 = new MetadataDelta.Builder() + .setImage(image) + .build() +delta2.replay(new RemoveAccessControlEntryRecord().setId(acl3.id())) +val image2 = delta2.apply(MetadataProvenance.EMPTY) +kraftMigrationZkWriter.handleDelta(image, image2, delta2, (_, _, operation) => { + migrationState = operation.apply(migrationState) +}) + +// verify the other 2 ACLs are still in ZK +val aclsInZk2 = zkClient.getVersionedAclsForResource(resource).acls +assertEquals(2, aclsInZk2.size) + +// Add another ACL +val acl4 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("carol")) +delta2.replay(acl4) +val image3 = delta2.apply(MetadataProvenance.EMPTY) + +// This is a contrived error case. In practice, we will never pass the same image as prev and current.
(kafka) branch 3.6 updated: MINOR: fix scala compile issue (#15343)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new c4fa14a7d95 MINOR: fix scala compile issue (#15343) c4fa14a7d95 is described below commit c4fa14a7d95dbfc5db4c5476aa14cf013d33f2ca Author: David Arthur AuthorDate: Thu Feb 8 18:44:42 2024 -0500 MINOR: fix scala compile issue (#15343) Reviewers: David Jacot --- .../test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index bb4d3e646c7..a4045f57948 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -231,7 +231,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { // Sync image to ZK val errorLogs = mutable.Buffer[String]() -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, msg => errorLogs.append(msg)) kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { migrationState = operation.apply(migrationState) }) @@ -276,7 +276,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { def testAclUpdateAndDelete(): Unit = { zkClient.createAclPaths() val errorLogs = mutable.Buffer[String]() -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, msg => errorLogs.append(msg)) val topicName = "topic-" + Uuid.randomUuid() val otherName = "other-" + Uuid.randomUuid()
(kafka) branch 3.7 updated: MINOR: fix scala compile issue (#15343)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 38bac936343 MINOR: fix scala compile issue (#15343) 38bac936343 is described below commit 38bac936343cc6a03d0f558b7ae8917cdb2707b7 Author: David Arthur AuthorDate: Thu Feb 8 18:44:42 2024 -0500 MINOR: fix scala compile issue (#15343) Reviewers: David Jacot --- .../test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index bb4d3e646c7..a4045f57948 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -231,7 +231,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { // Sync image to ZK val errorLogs = mutable.Buffer[String]() -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, msg => errorLogs.append(msg)) kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { migrationState = operation.apply(migrationState) }) @@ -276,7 +276,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { def testAclUpdateAndDelete(): Unit = { zkClient.createAclPaths() val errorLogs = mutable.Buffer[String]() -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, msg => errorLogs.append(msg)) val topicName = "topic-" + Uuid.randomUuid() val otherName = "other-" + Uuid.randomUuid()
(kafka) branch trunk updated: MINOR: fix scala compile issue (#15343)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 116bc000c8c MINOR: fix scala compile issue (#15343) 116bc000c8c is described below commit 116bc000c8c6533552321fe1d395629c2aa00bd9 Author: David Arthur AuthorDate: Thu Feb 8 18:44:42 2024 -0500 MINOR: fix scala compile issue (#15343) Reviewers: David Jacot --- .../test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index bb4d3e646c7..a4045f57948 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -231,7 +231,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { // Sync image to ZK val errorLogs = mutable.Buffer[String]() -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, msg => errorLogs.append(msg)) kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { migrationState = operation.apply(migrationState) }) @@ -276,7 +276,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { def testAclUpdateAndDelete(): Unit = { zkClient.createAclPaths() val errorLogs = mutable.Buffer[String]() -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, msg => errorLogs.append(msg)) val topicName = "topic-" + Uuid.randomUuid() val otherName = "other-" + Uuid.randomUuid()
(kafka) branch 3.6 updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new 3c623b4fb50 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) 3c623b4fb50 is described below commit 3c623b4fb50144cba782c0704f345e6473e66f02 Author: David Arthur AuthorDate: Wed Feb 7 15:54:59 2024 -0500 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) Reviewers: Colin P. McCabe --- .../zk/migration/ZkAclMigrationClientTest.scala| 162 - .../zk/migration/ZkConfigMigrationClientTest.scala | 4 +- .../kafka/zk/migration/ZkMigrationClientTest.scala | 2 +- .../java/org/apache/kafka/image/AclsDelta.java | 13 -- .../metadata/migration/KRaftMigrationDriver.java | 5 +- .../metadata/migration/KRaftMigrationZkWriter.java | 70 + .../migration/KRaftMigrationZkWriterTest.java | 6 +- 7 files changed, 206 insertions(+), 56 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index 0107b9fa954..bb4d3e646c7 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -21,14 +21,14 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString import kafka.utils.TestUtils import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ -import org.apache.kafka.common.metadata.AccessControlEntryRecord +import org.apache.kafka.common.metadata.{AccessControlEntryRecord, RemoveAccessControlEntryRecord} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter import org.apache.kafka.server.common.ApiMessageAndVersion -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail} import org.junit.jupiter.api.Test import scala.collection.mutable @@ -169,7 +169,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { val image = delta.apply(MetadataProvenance.EMPTY) // load snapshot to Zookeeper. -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, fail(_)) kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { migrationState = operation.apply(migrationState) }) // Verify the new ACLs in Zookeeper. @@ -189,4 +189,160 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { AclPermissionType.fromCode(acl1Resource3.permissionType())), resource3AclsInZk.head.ace) } + + def user(user: String): String = { +new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString + } + + def acl(resourceName: String, + resourceType: ResourceType, + resourcePattern: PatternType, + principal: String, + host: String = "*", + operation: AclOperation = AclOperation.READ, + permissionType: AclPermissionType = AclPermissionType.ALLOW + ): AccessControlEntryRecord = { +new AccessControlEntryRecord() + .setId(Uuid.randomUuid()) + .setHost(host) + .setOperation(operation.code()) + .setPrincipal(principal) + .setPermissionType(permissionType.code()) + .setPatternType(resourcePattern.code()) + .setResourceName(resourceName) + .setResourceType(resourceType.code()) + } + + @Test + def testDeleteOneAclOfMany(): Unit = { +zkClient.createAclPaths() +val topicName = "topic-" + Uuid.randomUuid() +val resource = new ResourcePattern(ResourceType.TOPIC, topicName, PatternType.LITERAL) + +// Create a delta with some ACLs +val delta = new MetadataDelta(MetadataImage.EMPTY) +val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("alice")) +val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("bob")) +val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("carol")) +delta.replay(acl1) +delta.replay(acl2) +delta.replay(acl3) +val image = delta.apply(MetadataProvenance.EMPTY) + +// Sync image to ZK +val errorLogs = mutable.Buffer[String]() +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +kraftMigrationZkWriter.handleSnapshot(image, (_, _, op
(kafka) branch 3.7 updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new b0721296b25 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) b0721296b25 is described below commit b0721296b259aebbbd80bbd10597a46e55b32655 Author: David Arthur AuthorDate: Wed Feb 7 15:54:59 2024 -0500 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) Reviewers: Colin P. McCabe --- .../zk/migration/ZkAclMigrationClientTest.scala| 162 - .../zk/migration/ZkConfigMigrationClientTest.scala | 4 +- .../kafka/zk/migration/ZkMigrationClientTest.scala | 2 +- .../java/org/apache/kafka/image/AclsDelta.java | 13 -- .../metadata/migration/KRaftMigrationDriver.java | 5 +- .../metadata/migration/KRaftMigrationZkWriter.java | 70 + .../migration/KRaftMigrationZkWriterTest.java | 6 +- 7 files changed, 206 insertions(+), 56 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index 0107b9fa954..bb4d3e646c7 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -21,14 +21,14 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString import kafka.utils.TestUtils import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ -import org.apache.kafka.common.metadata.AccessControlEntryRecord +import org.apache.kafka.common.metadata.{AccessControlEntryRecord, RemoveAccessControlEntryRecord} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter import org.apache.kafka.server.common.ApiMessageAndVersion -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail} import org.junit.jupiter.api.Test import scala.collection.mutable @@ -169,7 +169,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { val image = delta.apply(MetadataProvenance.EMPTY) // load snapshot to Zookeeper. -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, fail(_)) kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { migrationState = operation.apply(migrationState) }) // Verify the new ACLs in Zookeeper. @@ -189,4 +189,160 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { AclPermissionType.fromCode(acl1Resource3.permissionType())), resource3AclsInZk.head.ace) } + + def user(user: String): String = { +new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString + } + + def acl(resourceName: String, + resourceType: ResourceType, + resourcePattern: PatternType, + principal: String, + host: String = "*", + operation: AclOperation = AclOperation.READ, + permissionType: AclPermissionType = AclPermissionType.ALLOW + ): AccessControlEntryRecord = { +new AccessControlEntryRecord() + .setId(Uuid.randomUuid()) + .setHost(host) + .setOperation(operation.code()) + .setPrincipal(principal) + .setPermissionType(permissionType.code()) + .setPatternType(resourcePattern.code()) + .setResourceName(resourceName) + .setResourceType(resourceType.code()) + } + + @Test + def testDeleteOneAclOfMany(): Unit = { +zkClient.createAclPaths() +val topicName = "topic-" + Uuid.randomUuid() +val resource = new ResourcePattern(ResourceType.TOPIC, topicName, PatternType.LITERAL) + +// Create a delta with some ACLs +val delta = new MetadataDelta(MetadataImage.EMPTY) +val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("alice")) +val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("bob")) +val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("carol")) +delta.replay(acl1) +delta.replay(acl2) +delta.replay(acl3) +val image = delta.apply(MetadataProvenance.EMPTY) + +// Sync image to ZK +val errorLogs = mutable.Buffer[String]() +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +kraftMigrationZkWriter.handleSnapshot(image, (_, _, op
(kafka) branch trunk updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new c000b1fae2b MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) c000b1fae2b is described below commit c000b1fae2bd7d4b76713a53508f128a13431ab6 Author: David Arthur AuthorDate: Wed Feb 7 15:54:59 2024 -0500 MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327) Reviewers: Colin P. McCabe --- .../zk/migration/ZkAclMigrationClientTest.scala| 162 - .../zk/migration/ZkConfigMigrationClientTest.scala | 4 +- .../kafka/zk/migration/ZkMigrationClientTest.scala | 2 +- .../java/org/apache/kafka/image/AclsDelta.java | 13 -- .../metadata/migration/KRaftMigrationDriver.java | 5 +- .../metadata/migration/KRaftMigrationZkWriter.java | 70 + .../migration/KRaftMigrationZkWriterTest.java | 6 +- 7 files changed, 206 insertions(+), 56 deletions(-) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala index 0107b9fa954..bb4d3e646c7 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala @@ -21,14 +21,14 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString import kafka.utils.TestUtils import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ -import org.apache.kafka.common.metadata.AccessControlEntryRecord +import org.apache.kafka.common.metadata.{AccessControlEntryRecord, RemoveAccessControlEntryRecord} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.SecurityUtils import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter import org.apache.kafka.server.common.ApiMessageAndVersion -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail} import org.junit.jupiter.api.Test import scala.collection.mutable @@ -169,7 +169,7 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { val image = delta.apply(MetadataProvenance.EMPTY) // load snapshot to Zookeeper. -val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient) +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, fail(_)) kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { migrationState = operation.apply(migrationState) }) // Verify the new ACLs in Zookeeper. @@ -189,4 +189,160 @@ class ZkAclMigrationClientTest extends ZkMigrationTestHarness { AclPermissionType.fromCode(acl1Resource3.permissionType())), resource3AclsInZk.head.ace) } + + def user(user: String): String = { +new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString + } + + def acl(resourceName: String, + resourceType: ResourceType, + resourcePattern: PatternType, + principal: String, + host: String = "*", + operation: AclOperation = AclOperation.READ, + permissionType: AclPermissionType = AclPermissionType.ALLOW + ): AccessControlEntryRecord = { +new AccessControlEntryRecord() + .setId(Uuid.randomUuid()) + .setHost(host) + .setOperation(operation.code()) + .setPrincipal(principal) + .setPermissionType(permissionType.code()) + .setPatternType(resourcePattern.code()) + .setResourceName(resourceName) + .setResourceType(resourceType.code()) + } + + @Test + def testDeleteOneAclOfMany(): Unit = { +zkClient.createAclPaths() +val topicName = "topic-" + Uuid.randomUuid() +val resource = new ResourcePattern(ResourceType.TOPIC, topicName, PatternType.LITERAL) + +// Create a delta with some ACLs +val delta = new MetadataDelta(MetadataImage.EMPTY) +val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("alice")) +val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("bob")) +val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, user("carol")) +delta.replay(acl1) +delta.replay(acl2) +delta.replay(acl3) +val image = delta.apply(MetadataProvenance.EMPTY) + +// Sync image to ZK +val errorLogs = mutable.Buffer[String]() +val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, errorLogs.append) +kraftMigrationZkWriter.handleSnapshot(image, (_, _, op
(kafka) branch 3.7 updated: KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 5e9c61d79e7 KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293) 5e9c61d79e7 is described below commit 5e9c61d79e79960f138b6020450f888947217382 Author: Colin Patrick McCabe AuthorDate: Fri Feb 2 15:49:10 2024 -0800 KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293) While migrating from ZK mode to KRaft mode, the broker passes through a "hybrid" phase, in which it receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft controller. For the most part, these RPCs can be handled just like their traditional equivalents from a ZK-based controller. However, there is one thing that is different: the way topic deletions are handled. In ZK mode, there is a "deleting" state which topics enter prior to being completely removed. Partitions stay in this state until they are removed from the disks of all replicas. And partitions associated with these deleting topics show up in the UMR and LAIR as having a leader of -2 (which is not a valid broker ID, of course, because it's negative). When brokers receive these RPCs, they know to remove the associated partitions from their metadata caches, and disks. When a full UMR or ISR is sent, deleting partitions are included as well. In hybrid mode, in contrast, there is no "deleting" state. Topic deletion happens immediately. We can do this because we know that we have topic IDs that are never reused. This means that we can always tell the difference between a broker that had an old version of some topic, and a broker that has a new version that was re-created with the same name. To make this work, when handling a full UMR or LAIR, hybrid brokers must compare the full state that was sent over the wire to their own local state, and adjust accordingly. Prior to this PR, the code for handling those adjustments had several major flaws. The biggest flaw is that it did not correctly handle the "re-creation" case where a topic named FOO appears in the RPC, but with a different ID than the broker's local FOO. Another flaw is that a problem with a single partition would prevent handling the whole request. In ZkMetadataCache.scala, we handle full UMR requests from KRaft controllers by rewriting the UMR so that it contains the implied deletions. I fixed this code so that deletions always appear at the start of the list of topic states. This is important for the re-creation case since it means that a single request can both delete the old FOO and add a new FOO to the cache. Also, rather than modifying the requesst in-place, as the previous code did, I build a whole new request with the desired list of topic states. This is much safer because it avoids unforseen interactions with other parts of the code that deal with requests (like request logging). While this new copy may sound expensive, it should actually not be. We are doing a "shallow copy" which references the previous list topic state entries. I also reworked ZkMetadataCache.updateMetadata so that if a partition is re-created, it does not appear in the returned set of deleted TopicPartitions. Since this set is used only by the group manager, this seemed appropriate. (If I was in the consumer group for the previous iteration of FOO, I should still be in the consumer group for the new iteration.) On the ReplicaManager.scala side, we handle full LAIR requests by treating anything which does not appear in them as a "stray replica." (But we do not rewrite the request objects as we do with UMR.) I moved the logic for finding stray replicas from ReplicaManager into LogManager. It makes more sense there, since the information about what is on-disk is managed in LogManager. Also, the stray replica detection logic for KRaft mode is there, so it makes sense to put the stray replica detection logic for hybrid mode there as well. Since the stray replica detection is now in LogManager, I moved the unit tests there as well. Previously some of those tests had been in BrokerMetadataPublisherTest for historical reasons. The main advantage of the new LAIR logic is that it takes topic ID into account. A replica can be a stray even if the LAIR contains a topic of the given name, but a different ID. I also moved the stray replica handling earlier in the becomeLeaderOrFollower function, so that we could correctly handle the "delete and re-create FOO" case. Reviewers: David Arthur --- .../kafka/common/requests/LeaderAndIs
(kafka) branch trunk updated: KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 4169ac9f5dc KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293) 4169ac9f5dc is described below commit 4169ac9f5dc819c0495247c2230e96581800b1c7 Author: Colin Patrick McCabe AuthorDate: Fri Feb 2 15:49:10 2024 -0800 KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293) While migrating from ZK mode to KRaft mode, the broker passes through a "hybrid" phase, in which it receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft controller. For the most part, these RPCs can be handled just like their traditional equivalents from a ZK-based controller. However, there is one thing that is different: the way topic deletions are handled. In ZK mode, there is a "deleting" state which topics enter prior to being completely removed. Partitions stay in this state until they are removed from the disks of all replicas. And partitions associated with these deleting topics show up in the UMR and LAIR as having a leader of -2 (which is not a valid broker ID, of course, because it's negative). When brokers receive these RPCs, they know to remove the associated partitions from their metadata caches, and disks. When a full UMR or ISR is sent, deleting partitions are included as well. In hybrid mode, in contrast, there is no "deleting" state. Topic deletion happens immediately. We can do this because we know that we have topic IDs that are never reused. This means that we can always tell the difference between a broker that had an old version of some topic, and a broker that has a new version that was re-created with the same name. To make this work, when handling a full UMR or LAIR, hybrid brokers must compare the full state that was sent over the wire to their own local state, and adjust accordingly. Prior to this PR, the code for handling those adjustments had several major flaws. The biggest flaw is that it did not correctly handle the "re-creation" case where a topic named FOO appears in the RPC, but with a different ID than the broker's local FOO. Another flaw is that a problem with a single partition would prevent handling the whole request. In ZkMetadataCache.scala, we handle full UMR requests from KRaft controllers by rewriting the UMR so that it contains the implied deletions. I fixed this code so that deletions always appear at the start of the list of topic states. This is important for the re-creation case since it means that a single request can both delete the old FOO and add a new FOO to the cache. Also, rather than modifying the requesst in-place, as the previous code did, I build a whole new request with the desired list of topic states. This is much safer because it avoids unforseen interactions with other parts of the code that deal with requests (like request logging). While this new copy may sound expensive, it should actually not be. We are doing a "shallow copy" which references the previous list topic state entries. I also reworked ZkMetadataCache.updateMetadata so that if a partition is re-created, it does not appear in the returned set of deleted TopicPartitions. Since this set is used only by the group manager, this seemed appropriate. (If I was in the consumer group for the previous iteration of FOO, I should still be in the consumer group for the new iteration.) On the ReplicaManager.scala side, we handle full LAIR requests by treating anything which does not appear in them as a "stray replica." (But we do not rewrite the request objects as we do with UMR.) I moved the logic for finding stray replicas from ReplicaManager into LogManager. It makes more sense there, since the information about what is on-disk is managed in LogManager. Also, the stray replica detection logic for KRaft mode is there, so it makes sense to put the stray replica detection logic for hybrid mode there as well. Since the stray replica detection is now in LogManager, I moved the unit tests there as well. Previously some of those tests had been in BrokerMetadataPublisherTest for historical reasons. The main advantage of the new LAIR logic is that it takes topic ID into account. A replica can be a stray even if the LAIR contains a topic of the given name, but a different ID. I also moved the stray replica handling earlier in the becomeLeaderOrFollower function, so that we could correctly handle the "delete and re-create FOO" case. Reviewers: David Arthur --- .../kafka/
(kafka) branch 3.7 updated: KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 73fb4de9aa0 KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262) 73fb4de9aa0 is described below commit 73fb4de9aa03b2c39b11d7c296e55154a3b341de Author: Gaurav Narula AuthorDate: Fri Feb 2 17:47:14 2024 + KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262) In KRaft mode, or on ZK brokers that are migrating to KRaft, we have a local __cluster_metadata log. This log is stored in a single log directory which is configured via metadata.log.dir. If there is no metadata.log.dir given, it defaults to the first entry in log.dirs. In the future we may support multiple metadata log directories, but we don't yet. For now, we must abort the process when this log directory fails. In ZK mode, it is not necessary to abort the process when this directory fails, since there is no __cluster_metadata log there. This PR changes the logic so that we check for whether we're in ZK mode and do not abort in that scenario (unless we lost the final remaining log directory. of course.) Reviewers: Luke Chen , Colin P. McCabe , Omnia G H Ibrahim , Proven Provenzano --- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../unit/kafka/server/ReplicaManagerTest.scala | 64 +- 2 files changed, 64 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e445fb7215a..2c6958cc34a 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2598,7 +2598,7 @@ class ReplicaManager(val config: KafkaConfig, // retrieve the UUID here because logManager.handleLogDirFailure handler removes it val uuid = logManager.directoryId(dir) logManager.handleLogDirFailure(dir) -if (dir == config.metadataLogDir) { +if (dir == new File(config.metadataLogDir).getAbsolutePath && (config.processRoles.nonEmpty || config.migrationEnabled)) { fatal(s"Shutdown broker because the metadata log dir $dir has failed") Exit.halt(1) } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 9cc283eecad..4eab4df73c0 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -33,7 +33,7 @@ import kafka.log._ import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile} import kafka.server.epoch.util.MockBlockingSender -import kafka.utils.{Pool, TestInfoUtils, TestUtils} +import kafka.utils.{Exit, Pool, TestInfoUtils, TestUtils} import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.errors.{InvalidPidMappingException, KafkaStorageException} import org.apache.kafka.common.message.LeaderAndIsrRequestData @@ -71,6 +71,7 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import com.yammer.metrics.core.{Gauge, Meter} import kafka.log.remote.RemoteLogManager +import kafka.zk.KafkaZkClient import org.apache.kafka.common.config.{AbstractConfig, TopicConfig} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.raft.RaftConfig @@ -6369,6 +6370,67 @@ class ReplicaManagerTest { replicaManager.shutdown(checkpointHW = false) } } + + @Test + def testMetadataLogDirFailureInZkShouldNotHaltBroker(): Unit = { +// Given +val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, logDirCount = 2) +val config = KafkaConfig.fromProps(props) +val logDirFiles = config.logDirs.map(new File(_)) +val logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) +val logManager = TestUtils.createLogManager(logDirFiles, defaultConfig = new LogConfig(new Properties()), time = time) +val mockZkClient = mock(classOf[KafkaZkClient]) +val replicaManager = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = time.scheduler, + logManager = logManager, + quotaManagers = quotaManager, + metadataCache = MetadataCache.zkMetadataCache(config.brokerId, config.interBrokerProtocolVersion), + logDirFailureChannel = logDirFailureChannel, + alterPartitionManager = alterPartitionManager, + threadNamePrefix = Option(this.getClass.getName), + zkClient = Option(mockZkClient), +) + +try { +
(kafka) branch trunk updated: KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 3d95a69a28c KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262) 3d95a69a28c is described below commit 3d95a69a28c2d16e96618cfa9a1eb69180fb66ea Author: Gaurav Narula AuthorDate: Fri Feb 2 17:47:14 2024 + KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262) In KRaft mode, or on ZK brokers that are migrating to KRaft, we have a local __cluster_metadata log. This log is stored in a single log directory which is configured via metadata.log.dir. If there is no metadata.log.dir given, it defaults to the first entry in log.dirs. In the future we may support multiple metadata log directories, but we don't yet. For now, we must abort the process when this log directory fails. In ZK mode, it is not necessary to abort the process when this directory fails, since there is no __cluster_metadata log there. This PR changes the logic so that we check for whether we're in ZK mode and do not abort in that scenario (unless we lost the final remaining log directory. of course.) Reviewers: Luke Chen , Colin P. McCabe , Omnia G H Ibrahim , Proven Provenzano --- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../unit/kafka/server/ReplicaManagerTest.scala | 64 +- 2 files changed, 64 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 66ac417b5b1..5f34ade6021 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2453,7 +2453,7 @@ class ReplicaManager(val config: KafkaConfig, // retrieve the UUID here because logManager.handleLogDirFailure handler removes it val uuid = logManager.directoryId(dir) logManager.handleLogDirFailure(dir) -if (dir == config.metadataLogDir) { +if (dir == new File(config.metadataLogDir).getAbsolutePath && (config.processRoles.nonEmpty || config.migrationEnabled)) { fatal(s"Shutdown broker because the metadata log dir $dir has failed") Exit.halt(1) } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 174d4192fe4..4ec1a392c1f 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -33,7 +33,7 @@ import kafka.log._ import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile} import kafka.server.epoch.util.MockBlockingSender -import kafka.utils.{Pool, TestInfoUtils, TestUtils} +import kafka.utils.{Exit, Pool, TestInfoUtils, TestUtils} import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.errors.{InvalidPidMappingException, KafkaStorageException} import org.apache.kafka.common.message.LeaderAndIsrRequestData @@ -71,6 +71,7 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import com.yammer.metrics.core.{Gauge, Meter} import kafka.log.remote.RemoteLogManager +import kafka.zk.KafkaZkClient import org.apache.kafka.common.config.{AbstractConfig, TopicConfig} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.raft.RaftConfig @@ -6361,6 +6362,67 @@ class ReplicaManagerTest { replicaManager.shutdown(checkpointHW = false) } } + + @Test + def testMetadataLogDirFailureInZkShouldNotHaltBroker(): Unit = { +// Given +val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, logDirCount = 2) +val config = KafkaConfig.fromProps(props) +val logDirFiles = config.logDirs.map(new File(_)) +val logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) +val logManager = TestUtils.createLogManager(logDirFiles, defaultConfig = new LogConfig(new Properties()), time = time) +val mockZkClient = mock(classOf[KafkaZkClient]) +val replicaManager = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = time.scheduler, + logManager = logManager, + quotaManagers = quotaManager, + metadataCache = MetadataCache.zkMetadataCache(config.brokerId, config.interBrokerProtocolVersion), + logDirFailureChannel = logDirFailureChannel, + alterPartitionManager = alterPartitionManager, + threadNamePrefix = Option(this.getClass.getName), + zkClient = Option(mockZkClient), +) + +try { +
(kafka) branch trunk updated: Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)"
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new b8ad609541b Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)" b8ad609541b is described below commit b8ad609541bb8b224d16ce832a08f51017002bc1 Author: Colin P. McCabe AuthorDate: Thu Feb 1 16:17:18 2024 -0800 Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)" This reverts commit 46d76269fa72020d3d56d9e41d6f337563152837. --- docs/ops.html | 11 +-- 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index eb8c7431665..8cb65549db6 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4045,7 +4045,7 @@ listeners=CONTROLLER://:9093 On each broker, remove the zookeeper.metadata.migration.enable, controller.listener.names, and controller.quorum.voters - configurations. + configurations, and replace node.id with broker.id. Then perform a rolling restart of all brokers. @@ -4054,7 +4054,7 @@ listeners=CONTROLLER://:9093 - It is important to perform the zookeeper-shell.sh step quickly together with rolling at least one broker (into pure ZooKeeper mode), to minimize the amount of + It is important to perform the zookeeper-shell.sh step quickly, to minimize the amount of time that the cluster lacks a controller. @@ -4063,8 +4063,7 @@ listeners=CONTROLLER://:9093 - On each broker, remove the process.roles configuration, - replace the node.id with broker.id and + On each broker, remove the process.roles configuration, and restore the zookeeper.connect configuration to its previous value. If your cluster requires other ZooKeeper configurations for brokers, such as zookeeper.ssl.protocol, re-add those configurations as well. @@ -4080,7 +4079,7 @@ listeners=CONTROLLER://:9093 On each broker, remove the zookeeper.metadata.migration.enable, controller.listener.names, and controller.quorum.voters - configurations. + configurations. Replace node.id with broker.id. Then perform a second rolling restart of all brokers. @@ -4091,7 +4090,7 @@ listeners=CONTROLLER://:9093 - It is important to perform the zookeeper-shell.sh step quickly together with rolling at least one broker (into pure ZooKeeper mode), to minimize the amount of + It is important to perform the zookeeper-shell.sh step quickly, to minimize the amount of time that the cluster lacks a controller.
(kafka) branch 3.7 updated: Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)"
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new a1addb5668b Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)" a1addb5668b is described below commit a1addb5668b146f3cc77b61bba9e8b6ad4e5d9a3 Author: Colin P. McCabe AuthorDate: Thu Feb 1 16:16:43 2024 -0800 Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)" This reverts commit f7882a2cda5117ddc7f3650c9f3655bcc89b2671. --- docs/ops.html | 11 +-- 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index eb8c7431665..8cb65549db6 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4045,7 +4045,7 @@ listeners=CONTROLLER://:9093 On each broker, remove the zookeeper.metadata.migration.enable, controller.listener.names, and controller.quorum.voters - configurations. + configurations, and replace node.id with broker.id. Then perform a rolling restart of all brokers. @@ -4054,7 +4054,7 @@ listeners=CONTROLLER://:9093 - It is important to perform the zookeeper-shell.sh step quickly together with rolling at least one broker (into pure ZooKeeper mode), to minimize the amount of + It is important to perform the zookeeper-shell.sh step quickly, to minimize the amount of time that the cluster lacks a controller. @@ -4063,8 +4063,7 @@ listeners=CONTROLLER://:9093 - On each broker, remove the process.roles configuration, - replace the node.id with broker.id and + On each broker, remove the process.roles configuration, and restore the zookeeper.connect configuration to its previous value. If your cluster requires other ZooKeeper configurations for brokers, such as zookeeper.ssl.protocol, re-add those configurations as well. @@ -4080,7 +4079,7 @@ listeners=CONTROLLER://:9093 On each broker, remove the zookeeper.metadata.migration.enable, controller.listener.names, and controller.quorum.voters - configurations. + configurations. Replace node.id with broker.id. Then perform a second rolling restart of all brokers. @@ -4091,7 +4090,7 @@ listeners=CONTROLLER://:9093 - It is important to perform the zookeeper-shell.sh step quickly together with rolling at least one broker (into pure ZooKeeper mode), to minimize the amount of + It is important to perform the zookeeper-shell.sh step quickly, to minimize the amount of time that the cluster lacks a controller.
(kafka) branch 3.7 updated: KAFKA-16216: Reduce batch size for initial metadata load during ZK migration
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 945f4b91dfc KAFKA-16216: Reduce batch size for initial metadata load during ZK migration 945f4b91dfc is described below commit 945f4b91dfc2dd8ebfa49f04fba6230d01879d05 Author: David Arthur AuthorDate: Thu Feb 1 15:29:07 2024 -0800 KAFKA-16216: Reduce batch size for initial metadata load during ZK migration During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but 200 seems better. This PR also adds an internal configuration to control this batch size, for testing purposes. Reviewers: Colin P. McCabe --- .../main/scala/kafka/server/ControllerServer.scala | 1 + core/src/main/scala/kafka/server/KafkaConfig.scala | 5 + .../metadata/migration/KRaftMigrationDriver.java | 17 +++--- .../migration/KRaftMigrationDriverTest.java| 26 +- 4 files changed, 36 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index fb7439d8e56..f229b1008aa 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -302,6 +302,7 @@ class ControllerServer( .setQuorumFeatures(quorumFeatures) .setConfigSchema(configSchema) .setControllerMetrics(quorumControllerMetrics) + .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize) .setTime(time) .build() migrationDriver.start() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index fbd27114c61..5ae8b236d04 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -92,6 +92,7 @@ object Defaults { /** KRaft mode configs */ val EmptyNodeId: Int = -1 val ServerMaxStartupTimeMs = Long.MaxValue + val MigrationMetadataMinBatchSize = 200 /* Authorizer Configuration ***/ val AuthorizerClassName = "" @@ -403,6 +404,7 @@ object KafkaConfig { /** ZK to KRaft Migration configs */ val MigrationEnabledProp = "zookeeper.metadata.migration.enable" + val MigrationMetadataMinBatchSizeProp = "zookeeper.metadata.migration.min.batch.size" /** Enable eligible leader replicas configs */ val ElrEnabledProp = "eligible.leader.replicas.enable" @@ -1260,6 +1262,8 @@ object KafkaConfig { .defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.ServerMaxStartupTimeMs, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") .define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas") + .defineInternal(MigrationMetadataMinBatchSizeProp, INT, Defaults.MigrationMetadataMinBatchSize, atLeast(1), +MEDIUM, "Soft minimum batch size to use when migrating metadata from ZooKeeper to KRaft") /* Authorizer Configuration ***/ .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) @@ -1766,6 +1770,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp) + val migrationMetadataMinBatchSize: Int = getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp) val elrEnabled: Boolean = getBoolean(KafkaConfig.ElrEnabledProp) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 534d0940195..f33f0790e0a 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -90,8 +90,6 @@ public class KRaftMigrationDriver implements MetadataPublisher { */ final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000; -final static int MIGRATION_MIN_BATCH_SIZE = 1_000; - private final Time time; private final Logger log; private final int nodeId; @@ -110,6 +108,7 @@ public class KRaftMigrationDriver implements MetadataPublisher { * MetadataPublisher with MetadataLoader. */ private final Consumer initialZkLoadHandler; +private final int m
(kafka) branch trunk updated: KAFKA-16216: Reduce batch size for initial metadata load during ZK migration
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 12ce9c7f98c KAFKA-16216: Reduce batch size for initial metadata load during ZK migration 12ce9c7f98c is described below commit 12ce9c7f98c1617824d7bd86f9cc1f4560646e26 Author: David Arthur AuthorDate: Thu Feb 1 15:29:07 2024 -0800 KAFKA-16216: Reduce batch size for initial metadata load during ZK migration During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but 200 seems better. This PR also adds an internal configuration to control this batch size, for testing purposes. Reviewers: Colin P. McCabe --- .../main/scala/kafka/server/ControllerServer.scala | 1 + core/src/main/scala/kafka/server/KafkaConfig.scala | 4 .../metadata/migration/KRaftMigrationDriver.java | 17 +++--- .../migration/KRaftMigrationDriverTest.java| 26 +- .../org/apache/kafka/server/config/Defaults.java | 1 + 5 files changed, 36 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 8f14d814b63..a882e54e52d 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -304,6 +304,7 @@ class ControllerServer( .setQuorumFeatures(quorumFeatures) .setConfigSchema(configSchema) .setControllerMetrics(quorumControllerMetrics) + .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize) .setTime(time) .build() migrationDriver.start() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index f7967ed8457..c6f51a000e2 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -166,6 +166,7 @@ object KafkaConfig { /** ZK to KRaft Migration configs */ val MigrationEnabledProp = "zookeeper.metadata.migration.enable" + val MigrationMetadataMinBatchSizeProp = "zookeeper.metadata.migration.min.batch.size" /** Enable eligible leader replicas configs */ val ElrEnabledProp = "eligible.leader.replicas.enable" @@ -1029,6 +1030,8 @@ object KafkaConfig { .defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.SERVER_MAX_STARTUP_TIME_MS, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") .define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas") + .defineInternal(MigrationMetadataMinBatchSizeProp, INT, Defaults.MIGRATION_METADATA_MIN_BATCH_SIZE, atLeast(1), +MEDIUM, "Soft minimum batch size to use when migrating metadata from ZooKeeper to KRaft") /* Authorizer Configuration ***/ .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) @@ -1538,6 +1541,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp) + val migrationMetadataMinBatchSize: Int = getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp) val elrEnabled: Boolean = getBoolean(KafkaConfig.ElrEnabledProp) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 534d0940195..f33f0790e0a 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -90,8 +90,6 @@ public class KRaftMigrationDriver implements MetadataPublisher { */ final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000; -final static int MIGRATION_MIN_BATCH_SIZE = 1_000; - private final Time time; private final Logger log; private final int nodeId; @@ -110,6 +108,7 @@ public class KRaftMigrationDriver implements MetadataPublisher { * MetadataPublisher with MetadataLoader. */ private final Consumer initialZkLoadHandler; +private final int minBatchSize; private volatile MigrationDriverState migrationState; private volatile ZkMigrationLeadershipState migrationLeadershipState; private volatile MetadataImage image; @@ -
(kafka) branch 3.7 updated: KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new cf6defb8b53 KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2 cf6defb8b53 is described below commit cf6defb8b5327ae76d2e53467a8570261bad15b2 Author: Gaurav Narula AuthorDate: Tue Jan 30 09:59:11 2024 -0800 KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2 We update metadata update handler to resend broker registration when metadata has been updated to >= 3.7IV2 so that the controller becomes aware of the log directories in the broker. We also update DirectoryId::isOnline to return true on an empty list of log directories while the controller awaits broker registration. Co-authored-by: Proven Provenzano Reviewers: Omnia G H Ibrahim , Luke Chen , Colin P. McCabe --- .../kafka/server/BrokerLifecycleManager.scala | 13 +++ .../src/main/scala/kafka/server/BrokerServer.scala | 4 +- .../server/metadata/BrokerMetadataPublisher.scala | 15 ++- .../kafka/server/BrokerLifecycleManagerTest.scala | 32 ++ .../metadata/BrokerMetadataPublisherTest.scala | 111 - .../java/org/apache/kafka/common/DirectoryId.java | 10 ++ .../org/apache/kafka/common/DirectoryIdTest.java | 6 ++ 7 files changed, 184 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index c3da9dcb7e6..bd80364274d 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -244,6 +244,19 @@ class BrokerLifecycleManager( eventQueue.append(new OfflineDirEvent(directory)) } + def handleKraftJBODMetadataVersionUpdate(): Unit = { +eventQueue.append(new KraftJBODMetadataVersionUpdateEvent()) + } + + private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event { +override def run(): Unit = { + if (!isZkBroker) { +registered = false +scheduleNextCommunicationImmediately() + } +} + } + def brokerEpoch: Long = _brokerEpoch def state: BrokerState = _state diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 593d28ac7ab..462b147a162 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -452,7 +452,9 @@ class BrokerServer( authorizer ), sharedServer.initialBrokerMetadataLoadFaultHandler, -sharedServer.metadataPublishingFaultHandler) +sharedServer.metadataPublishingFaultHandler, +lifecycleManager + ) metadataPublishers.add(brokerMetadataPublisher) // Register parts of the broker that can be reconfigured via dynamic configs. This needs to diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 21bb95d9ecb..e49b910d246 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -20,7 +20,7 @@ package kafka.server.metadata import java.util.{OptionalInt, Properties} import kafka.coordinator.transaction.TransactionCoordinator import kafka.log.LogManager -import kafka.server.{KafkaConfig, ReplicaManager, RequestLocal} +import kafka.server.{BrokerLifecycleManager, KafkaConfig, ReplicaManager, RequestLocal} import kafka.utils.Logging import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.TimeoutException @@ -29,6 +29,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.fault.FaultHandler import java.util.concurrent.CompletableFuture @@ -72,7 +73,8 @@ class BrokerMetadataPublisher( delegationTokenPublisher: DelegationTokenPublisher, aclPublisher: AclPublisher, fatalFaultHandler: FaultHandler, - metadataPublishingFaultHandler: FaultHandler + metadataPublishingFaultHandler: FaultHandler, + brokerLifecycleManager: BrokerLifecycleManager, ) extends MetadataPublisher with Logging { logIdent = s"[BrokerMetadataPublisher id=${config.nodeId}] " @@ -130,6 +132,15 @@ class BrokerMetadataPublisher( Option(delta.featuresDelta()).foreach { featuresDelta => featuresDelta.metadataVersionChange().ifPresent{ metadataVersion =>
(kafka) branch trunk updated: KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 4c6f975ab3d KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2 4c6f975ab3d is described below commit 4c6f975ab3deba517c6621c4a051e9beb1240bbc Author: Gaurav Narula AuthorDate: Tue Jan 30 09:59:11 2024 -0800 KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2 We update metadata update handler to resend broker registration when metadata has been updated to >= 3.7IV2 so that the controller becomes aware of the log directories in the broker. We also update DirectoryId::isOnline to return true on an empty list of log directories while the controller awaits broker registration. Co-authored-by: Proven Provenzano Reviewers: Omnia G H Ibrahim , Luke Chen , Colin P. McCabe --- .../kafka/server/BrokerLifecycleManager.scala | 13 +++ .../src/main/scala/kafka/server/BrokerServer.scala | 4 +- .../server/metadata/BrokerMetadataPublisher.scala | 15 ++- .../kafka/server/BrokerLifecycleManagerTest.scala | 32 ++ .../metadata/BrokerMetadataPublisherTest.scala | 111 - .../java/org/apache/kafka/common/DirectoryId.java | 10 ++ .../org/apache/kafka/common/DirectoryIdTest.java | 6 ++ 7 files changed, 184 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index c3da9dcb7e6..bd80364274d 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -244,6 +244,19 @@ class BrokerLifecycleManager( eventQueue.append(new OfflineDirEvent(directory)) } + def handleKraftJBODMetadataVersionUpdate(): Unit = { +eventQueue.append(new KraftJBODMetadataVersionUpdateEvent()) + } + + private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event { +override def run(): Unit = { + if (!isZkBroker) { +registered = false +scheduleNextCommunicationImmediately() + } +} + } + def brokerEpoch: Long = _brokerEpoch def state: BrokerState = _state diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 60b6c554f16..6ee44e3070a 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -452,7 +452,9 @@ class BrokerServer( authorizer ), sharedServer.initialBrokerMetadataLoadFaultHandler, -sharedServer.metadataPublishingFaultHandler) +sharedServer.metadataPublishingFaultHandler, +lifecycleManager + ) metadataPublishers.add(brokerMetadataPublisher) // Register parts of the broker that can be reconfigured via dynamic configs. This needs to diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 663afd226de..8edb2a1aa1a 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -20,7 +20,7 @@ package kafka.server.metadata import java.util.{OptionalInt, Properties} import kafka.coordinator.transaction.TransactionCoordinator import kafka.log.LogManager -import kafka.server.{KafkaConfig, ReplicaManager, RequestLocal} +import kafka.server.{BrokerLifecycleManager, KafkaConfig, ReplicaManager, RequestLocal} import kafka.utils.Logging import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.TimeoutException @@ -29,6 +29,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.fault.FaultHandler import java.util.concurrent.CompletableFuture @@ -72,7 +73,8 @@ class BrokerMetadataPublisher( delegationTokenPublisher: DelegationTokenPublisher, aclPublisher: AclPublisher, fatalFaultHandler: FaultHandler, - metadataPublishingFaultHandler: FaultHandler + metadataPublishingFaultHandler: FaultHandler, + brokerLifecycleManager: BrokerLifecycleManager, ) extends MetadataPublisher with Logging { logIdent = s"[BrokerMetadataPublisher id=${config.nodeId}] " @@ -130,6 +132,15 @@ class BrokerMetadataPublisher( Option(delta.featuresDelta()).foreach { featuresDelta => featuresDelta.metadataVersionChange().ifPresent{ metadataVersion =>
(kafka) branch 3.7 updated: KAFKA-14616: Fix stray replica of recreated topics in KRaft mode
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 5a861075bdd KAFKA-14616: Fix stray replica of recreated topics in KRaft mode 5a861075bdd is described below commit 5a861075bdde71f786cbb71949c213449d62af27 Author: Colin P. McCabe AuthorDate: Mon Jan 29 22:36:09 2024 -0800 KAFKA-14616: Fix stray replica of recreated topics in KRaft mode When a broker is down, and a topic is deleted, this will result in that broker seeing "stray replicas" the next time it starts up. These replicas contain data that used to be important, but which now needs to be deleted. Stray replica deletion is handled during the initial metadata publishing step on the broker. Previously, we deleted these stray replicas after starting up BOTH LogManager and ReplicaManager. However, this wasn't quite correct. The presence of the stray replicas confused ReplicaManager. Instead, we should delete the stray replicas BEFORE starting ReplicaManager. This bug triggered when a topic was deleted and re-created while a broker was down, and some of the replicas of the re-created topic landed on that broker. The impact was that the stray replicas were deleted, but the new replicas for the next iteration of the topic never got created. This, in turn, led to persistent under-replication until the next time the broker was restarted. Reviewers: Luke Chen , Omnia G H Ibrahim , Gaurav Narula --- core/src/main/scala/kafka/log/LogManager.scala | 49 .../main/scala/kafka/server/ReplicaManager.scala | 12 .../server/metadata/BrokerMetadataPublisher.scala | 66 -- .../kafka/server/KRaftClusterTest.scala| 57 ++- .../metadata/BrokerMetadataPublisherTest.scala | 6 +- 5 files changed, 120 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index f84177d9717..48e739f6847 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.metadata.ConfigRepository import kafka.server._ +import kafka.server.metadata.BrokerMetadataPublisher.info import kafka.utils._ import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.utils.{KafkaThread, Time, Utils} @@ -35,6 +36,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Success, Try} import kafka.utils.Implicits._ import org.apache.kafka.common.config.TopicConfig +import org.apache.kafka.image.TopicsImage import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils} import java.util.{OptionalLong, Properties} @@ -563,6 +565,16 @@ class LogManager(logDirs: Seq[File], startupWithConfigOverrides(defaultConfig, fetchTopicConfigOverrides(defaultConfig, topicNames)) } + def deleteStrayKRaftReplicas( +brokerId: Int, +image: TopicsImage + ): Unit = { +val strayPartitions = findStrayReplicas(brokerId, image, allLogs) +strayPartitions.foreach(topicPartition => { + asyncDelete(topicPartition, false, false, true) +}) + } + // visible for testing @nowarn("cat=deprecation") private[log] def fetchTopicConfigOverrides(defaultConfig: LogConfig, topicNames: Set[String]): Map[String, LogConfig] = { @@ -1522,4 +1534,41 @@ object LogManager { remoteStorageSystemEnable = config.remoteLogManagerConfig.enableRemoteStorageSystem()) } + /** + * Find logs which should not be on the current broker, according to the metadata image. + * + * @param brokerIdThe ID of the current broker. + * @param newTopicsImage The new topics image after broker has been reloaded + * @param logsA collection of Log objects. + * + * @return The topic partitions which are no longer needed on this broker. + */ + def findStrayReplicas( +brokerId: Int, +newTopicsImage: TopicsImage, +logs: Iterable[UnifiedLog] + ): Iterable[TopicPartition] = { +logs.flatMap { log => + val topicId = log.topicId.getOrElse { +throw new RuntimeException(s"The log dir $log does not have a topic ID, " + + "which is not allowed when running in KRaft mode.") + } + + val partitionId = log.topicPartition.partition() + Option(newTopicsImage.getPartition(topicId, partitionId)) match { +case Some(partition) => + if (!partition.replicas.contains(brokerId)) { +info(s"Found stray log dir $log
(kafka) branch trunk updated: KAFKA-14616: Fix stray replica of recreated topics in KRaft mode
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new f7feb43af35 KAFKA-14616: Fix stray replica of recreated topics in KRaft mode f7feb43af35 is described below commit f7feb43af351efa25be37cb78814c19bd28a22a2 Author: Colin P. McCabe AuthorDate: Mon Jan 29 22:36:09 2024 -0800 KAFKA-14616: Fix stray replica of recreated topics in KRaft mode When a broker is down, and a topic is deleted, this will result in that broker seeing "stray replicas" the next time it starts up. These replicas contain data that used to be important, but which now needs to be deleted. Stray replica deletion is handled during the initial metadata publishing step on the broker. Previously, we deleted these stray replicas after starting up BOTH LogManager and ReplicaManager. However, this wasn't quite correct. The presence of the stray replicas confused ReplicaManager. Instead, we should delete the stray replicas BEFORE starting ReplicaManager. This bug triggered when a topic was deleted and re-created while a broker was down, and some of the replicas of the re-created topic landed on that broker. The impact was that the stray replicas were deleted, but the new replicas for the next iteration of the topic never got created. This, in turn, led to persistent under-replication until the next time the broker was restarted. Reviewers: Luke Chen , Omnia G H Ibrahim , Gaurav Narula --- core/src/main/scala/kafka/log/LogManager.scala | 49 .../main/scala/kafka/server/ReplicaManager.scala | 13 - .../server/metadata/BrokerMetadataPublisher.scala | 66 -- .../kafka/server/KRaftClusterTest.scala| 57 ++- .../metadata/BrokerMetadataPublisherTest.scala | 6 +- 5 files changed, 120 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index deb77c62d75..5e657ccf379 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.metadata.ConfigRepository import kafka.server._ +import kafka.server.metadata.BrokerMetadataPublisher.info import kafka.utils._ import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.utils.{KafkaThread, Time, Utils} @@ -35,6 +36,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Success, Try} import kafka.utils.Implicits._ import org.apache.kafka.common.config.TopicConfig +import org.apache.kafka.image.TopicsImage import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils} import java.util.{OptionalLong, Properties} @@ -563,6 +565,16 @@ class LogManager(logDirs: Seq[File], startupWithConfigOverrides(defaultConfig, fetchTopicConfigOverrides(defaultConfig, topicNames)) } + def deleteStrayKRaftReplicas( +brokerId: Int, +image: TopicsImage + ): Unit = { +val strayPartitions = findStrayReplicas(brokerId, image, allLogs) +strayPartitions.foreach(topicPartition => { + asyncDelete(topicPartition, false, false, true) +}) + } + // visible for testing @nowarn("cat=deprecation") private[log] def fetchTopicConfigOverrides(defaultConfig: LogConfig, topicNames: Set[String]): Map[String, LogConfig] = { @@ -1522,4 +1534,41 @@ object LogManager { remoteStorageSystemEnable = config.remoteLogManagerConfig.enableRemoteStorageSystem()) } + /** + * Find logs which should not be on the current broker, according to the metadata image. + * + * @param brokerIdThe ID of the current broker. + * @param newTopicsImage The new topics image after broker has been reloaded + * @param logsA collection of Log objects. + * + * @return The topic partitions which are no longer needed on this broker. + */ + def findStrayReplicas( +brokerId: Int, +newTopicsImage: TopicsImage, +logs: Iterable[UnifiedLog] + ): Iterable[TopicPartition] = { +logs.flatMap { log => + val topicId = log.topicId.getOrElse { +throw new RuntimeException(s"The log dir $log does not have a topic ID, " + + "which is not allowed when running in KRaft mode.") + } + + val partitionId = log.topicPartition.partition() + Option(newTopicsImage.getPartition(topicId, partitionId)) match { +case Some(partition) => + if (!partition.replicas.contains(brokerId)) { +info(s"Found stray log dir $log
(kafka) branch 3.7 updated: KAFKA-16101: Fix KRaft migration documentation (#15193)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new e71992d3bea KAFKA-16101: Fix KRaft migration documentation (#15193) e71992d3bea is described below commit e71992d3bea1913969e3750f884fae5483d39eef Author: Colin Patrick McCabe AuthorDate: Mon Jan 29 14:34:07 2024 -0800 KAFKA-16101: Fix KRaft migration documentation (#15193) This PR fixes some bugs in the KRaft migration documentation and reorganizes it to be easier to read. (Specifically, there were some steps that were previously out of order.) In order to keep it all straight, the revert documentation is now in the form of a table which maps the latest migration state to the actions which the system administrator should perform. Reviewers: Luke Chen , David Arthur , Liu Zeyu , Paolo Patierno --- docs/ops.html | 160 ++ 1 file changed, 139 insertions(+), 21 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index f943dbed630..8cb65549db6 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3874,7 +3874,7 @@ zookeeper.connect=localhost:2181 Note: The KRaft cluster node.id values must be different from any existing ZK broker broker.id. In KRaft-mode, the brokers and controllers share the same Node ID namespace. - Enabling the migration on the brokers + Enter Migration Mode on the Brokers Once the KRaft controller quorum has been started, the brokers will need to be reconfigured and restarted. Brokers may be restarted in a rolling fashion to avoid impacting cluster availability. Each broker requires the @@ -3919,9 +3919,10 @@ controller.listener.names=CONTROLLER Migrating brokers to KRaft -Once the KRaft controller completes the metadata migration, the brokers will still be running in ZK mode. While the -KRaft controller is in migration mode, it will continue sending controller RPCs to the ZK mode brokers. This includes -RPCs like UpdateMetadata and LeaderAndIsr. +Once the KRaft controller completes the metadata migration, the brokers will still be running +in ZooKeeper mode. While the KRaft controller is in migration mode, it will continue sending +controller RPCs to the ZooKeeper mode brokers. This includes RPCs like UpdateMetadata and +LeaderAndIsr. @@ -3956,29 +3957,16 @@ controller.listener.names=CONTROLLER Each broker is restarted with a KRaft configuration until the entire cluster is running in KRaft mode. - Reverting to ZooKeeper mode During the Migration -While the cluster is still in migration mode, it is possible to revert to ZK mode. In order to do this: - - -For each KRaft broker: - - Stop the broker. - Remove the __cluster_metadata directory on the broker. - Remove the zookeeper.metadata.migration.enable configuration and the KRaft controllers related configurations like controller.quorum.voters -and controller.listener.names from the broker configuration properties file. - Restart the broker in ZooKeeper mode. - - - Take down the KRaft quorum. - Using ZooKeeper shell, delete the controller node using rmr /controller, so that a ZooKeeper-based broker can become the next controller. - - Finalizing the migration Once all brokers have been restarted in KRaft mode, the last step to finalize the migration is to take the KRaft controllers out of migration mode. This is done by removing the "zookeeper.metadata.migration.enable" property from each of their configs and restarting them one at a time. + +Once the migration has been finalized, you can safely deprovision your ZooKeeper cluster, assuming you are +not using it for anything else. After this point, it is no longer possible to revert to ZooKeeper mode. + # Sample KRaft cluster controller.properties listening on 9093 @@ -3996,6 +3984,136 @@ listeners=CONTROLLER://:9093 # Other configs ... + Reverting to ZooKeeper mode During the Migration + +While the cluster is still in migration mode, it is possible to revert to ZooKeeper mode. The process +to follow depends on how far the migration has progressed. In order to find out how to revert, +select the final migration step that you have completed in this table. + + +Note that the directions given here assume that each step was fully completed, and they were +done in order. So, for example, we assume that if "Enter Migration Mode on the Brokers" was +completed, "Provisioning the KRaft controller quorum" was also fully completed previously. + + +If you did not fully complete any step, back out whatever you have done and then
(kafka) branch 3.7 updated: KAFKA-16171: Fix ZK migration controller race #15238
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new b7098722994 KAFKA-16171: Fix ZK migration controller race #15238 b7098722994 is described below commit b7098722994c7fc14ccc369b030123a8a6dc156b Author: David Arthur AuthorDate: Fri Jan 19 13:38:37 2024 -0500 KAFKA-16171: Fix ZK migration controller race #15238 This patch causes the active KRaftMigrationDriver to reload the /migration ZK state after electing itself as the leader in ZK. This closes a race condition where the previous active controller could make an update to /migration after the new leader was elected. The update race was not actually a problem regarding the data since both controllers would be syncing the same state from KRaft to ZK, but the change to the znode causes the new controller to fail on the zk version check on /migration. This patch also fixes a as-yet-unseen bug where the active controllers failing to elect itself via claimControllerLeadership would not retry. Reviewers: Colin P. McCabe --- .../kafka/zk/ZkMigrationFailoverTest.scala | 276 + .../metadata/migration/KRaftMigrationDriver.java | 14 +- .../migration/ZkMigrationLeadershipState.java | 9 +- .../migration/CapturingMigrationClient.java| 12 +- 4 files changed, 304 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala new file mode 100644 index 000..20dd5d91a83 --- /dev/null +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala @@ -0,0 +1,276 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.zk + +import kafka.utils.{Logging, PasswordEncoder, TestUtils} +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.metadata.{FeatureLevelRecord, TopicRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.controller.QuorumFeatures +import org.apache.kafka.controller.metrics.QuorumControllerMetrics +import org.apache.kafka.image.loader.LogDeltaManifest +import org.apache.kafka.image.publisher.MetadataPublisher +import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} +import org.apache.kafka.metadata.KafkaConfigSchema +import org.apache.kafka.metadata.migration._ +import org.apache.kafka.raft.{LeaderAndEpoch, OffsetAndEpoch} +import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.FaultHandler +import org.apache.zookeeper.client.ZKClientConfig +import org.junit.jupiter.api.Assertions.{assertTrue, fail} +import org.junit.jupiter.api.Test + +import java.util +import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.{Optional, OptionalInt} +import scala.collection.mutable + +class ZkMigrationFailoverTest extends Logging { + + class CapturingFaultHandler(nodeId: Int) extends FaultHandler { +val faults = mutable.Buffer[Throwable]() +var future: CompletableFuture[Throwable] = CompletableFuture.completedFuture(new RuntimeException()) +var waitingForMsg = "" + +override def handleFault(failureMessage: String, cause: Throwable): RuntimeException = { + error(s"Fault handled on node $nodeId", cause) + faults.append(cause) + if (!future.isDone && cause.getMessage.contains(waitingForMsg)) { +future.complete(cause) + } + new RuntimeException(cause) +} + +def checkAndClear(verifier: (Seq[Throwable]) => Unit): Unit = { + val faultsSoFar = faults.toSeq + try { +verifier.apply(faultsSoFar) + } catch { +case ae: AssertionError => fail(s"Assertion failed. Faults on $nodeId were: $faultsSoFar", ae) + } +} + +def waitForError(message: String): CompletableFuture[Throwable] = { + future = new CompletableFuture[Throwable]() + waitingForMsg = messag
(kafka) branch trunk updated: KAFKA-16101: Fix KRaft migration documentation (#15193)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 5b422613a66 KAFKA-16101: Fix KRaft migration documentation (#15193) 5b422613a66 is described below commit 5b422613a664740282440a5c65c7589095149d35 Author: Colin Patrick McCabe AuthorDate: Mon Jan 29 14:34:07 2024 -0800 KAFKA-16101: Fix KRaft migration documentation (#15193) This PR fixes some bugs in the KRaft migration documentation and reorganizes it to be easier to read. (Specifically, there were some steps that were previously out of order.) In order to keep it all straight, the revert documentation is now in the form of a table which maps the latest migration state to the actions which the system administrator should perform. Reviewers: Luke Chen , David Arthur , Liu Zeyu , Paolo Patierno --- docs/ops.html | 160 ++ 1 file changed, 139 insertions(+), 21 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index f943dbed630..8cb65549db6 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3874,7 +3874,7 @@ zookeeper.connect=localhost:2181 Note: The KRaft cluster node.id values must be different from any existing ZK broker broker.id. In KRaft-mode, the brokers and controllers share the same Node ID namespace. - Enabling the migration on the brokers + Enter Migration Mode on the Brokers Once the KRaft controller quorum has been started, the brokers will need to be reconfigured and restarted. Brokers may be restarted in a rolling fashion to avoid impacting cluster availability. Each broker requires the @@ -3919,9 +3919,10 @@ controller.listener.names=CONTROLLER Migrating brokers to KRaft -Once the KRaft controller completes the metadata migration, the brokers will still be running in ZK mode. While the -KRaft controller is in migration mode, it will continue sending controller RPCs to the ZK mode brokers. This includes -RPCs like UpdateMetadata and LeaderAndIsr. +Once the KRaft controller completes the metadata migration, the brokers will still be running +in ZooKeeper mode. While the KRaft controller is in migration mode, it will continue sending +controller RPCs to the ZooKeeper mode brokers. This includes RPCs like UpdateMetadata and +LeaderAndIsr. @@ -3956,29 +3957,16 @@ controller.listener.names=CONTROLLER Each broker is restarted with a KRaft configuration until the entire cluster is running in KRaft mode. - Reverting to ZooKeeper mode During the Migration -While the cluster is still in migration mode, it is possible to revert to ZK mode. In order to do this: - - -For each KRaft broker: - - Stop the broker. - Remove the __cluster_metadata directory on the broker. - Remove the zookeeper.metadata.migration.enable configuration and the KRaft controllers related configurations like controller.quorum.voters -and controller.listener.names from the broker configuration properties file. - Restart the broker in ZooKeeper mode. - - - Take down the KRaft quorum. - Using ZooKeeper shell, delete the controller node using rmr /controller, so that a ZooKeeper-based broker can become the next controller. - - Finalizing the migration Once all brokers have been restarted in KRaft mode, the last step to finalize the migration is to take the KRaft controllers out of migration mode. This is done by removing the "zookeeper.metadata.migration.enable" property from each of their configs and restarting them one at a time. + +Once the migration has been finalized, you can safely deprovision your ZooKeeper cluster, assuming you are +not using it for anything else. After this point, it is no longer possible to revert to ZooKeeper mode. + # Sample KRaft cluster controller.properties listening on 9093 @@ -3996,6 +3984,136 @@ listeners=CONTROLLER://:9093 # Other configs ... + Reverting to ZooKeeper mode During the Migration + +While the cluster is still in migration mode, it is possible to revert to ZooKeeper mode. The process +to follow depends on how far the migration has progressed. In order to find out how to revert, +select the final migration step that you have completed in this table. + + +Note that the directions given here assume that each step was fully completed, and they were +done in order. So, for example, we assume that if "Enter Migration Mode on the Brokers" was +completed, "Provisioning the KRaft controller quorum" was also fully completed previously. + + +If you did not fully complete any step, back out whatever you have done and then
(kafka) branch KAFKA-16101 deleted (was 52fbe9bd5c6)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git was 52fbe9bd5c6 KAFKA-16101: KRaft migration documentation is incorrect The revisions that were on this branch are still contained in other references; therefore, this change does not discard any commits from the repository.
(kafka) branch KAFKA-16101 updated (30cbfd1609f -> 52fbe9bd5c6)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git discard 30cbfd1609f fix discard 7690f2abed5 address review comments discard 6233d844c4a add table discard 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect add 52fbe9bd5c6 KAFKA-16101: KRaft migration documentation is incorrect This update added new revisions after undoing existing revisions. That is to say, some revisions that were in the old version of the branch are not in the new version. This situation occurs when a user --force pushes a change and generates a repository containing something like this: * -- * -- B -- O -- O -- O (30cbfd1609f) \ N -- N -- N refs/heads/KAFKA-16101 (52fbe9bd5c6) You should already have received notification emails for all of the O revisions, and so the following emails describe only the N revisions from the common base, B. Any revisions marked "omit" are not gone; other references still refer to them. Any revisions marked "discard" are gone forever. No new revisions were added by this update. Summary of changes: docs/ops.html | 32 1 file changed, 16 insertions(+), 16 deletions(-)
(kafka) branch trunk updated: KAFKA-16171: Fix ZK migration controller race #15238
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 16ed7357b18 KAFKA-16171: Fix ZK migration controller race #15238 16ed7357b18 is described below commit 16ed7357b1899d4711c93c1fa094369f8b92480b Author: David Arthur AuthorDate: Fri Jan 19 13:38:37 2024 -0500 KAFKA-16171: Fix ZK migration controller race #15238 This patch causes the active KRaftMigrationDriver to reload the /migration ZK state after electing itself as the leader in ZK. This closes a race condition where the previous active controller could make an update to /migration after the new leader was elected. The update race was not actually a problem regarding the data since both controllers would be syncing the same state from KRaft to ZK, but the change to the znode causes the new controller to fail on the zk version check on /migration. This patch also fixes a as-yet-unseen bug where the active controllers failing to elect itself via claimControllerLeadership would not retry. Reviewers: Colin P. McCabe --- .../kafka/zk/ZkMigrationFailoverTest.scala | 276 + .../metadata/migration/KRaftMigrationDriver.java | 14 +- .../migration/ZkMigrationLeadershipState.java | 9 +- .../migration/CapturingMigrationClient.java| 12 +- 4 files changed, 304 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala new file mode 100644 index 000..20dd5d91a83 --- /dev/null +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala @@ -0,0 +1,276 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.zk + +import kafka.utils.{Logging, PasswordEncoder, TestUtils} +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.metadata.{FeatureLevelRecord, TopicRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.controller.QuorumFeatures +import org.apache.kafka.controller.metrics.QuorumControllerMetrics +import org.apache.kafka.image.loader.LogDeltaManifest +import org.apache.kafka.image.publisher.MetadataPublisher +import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} +import org.apache.kafka.metadata.KafkaConfigSchema +import org.apache.kafka.metadata.migration._ +import org.apache.kafka.raft.{LeaderAndEpoch, OffsetAndEpoch} +import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.FaultHandler +import org.apache.zookeeper.client.ZKClientConfig +import org.junit.jupiter.api.Assertions.{assertTrue, fail} +import org.junit.jupiter.api.Test + +import java.util +import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.{Optional, OptionalInt} +import scala.collection.mutable + +class ZkMigrationFailoverTest extends Logging { + + class CapturingFaultHandler(nodeId: Int) extends FaultHandler { +val faults = mutable.Buffer[Throwable]() +var future: CompletableFuture[Throwable] = CompletableFuture.completedFuture(new RuntimeException()) +var waitingForMsg = "" + +override def handleFault(failureMessage: String, cause: Throwable): RuntimeException = { + error(s"Fault handled on node $nodeId", cause) + faults.append(cause) + if (!future.isDone && cause.getMessage.contains(waitingForMsg)) { +future.complete(cause) + } + new RuntimeException(cause) +} + +def checkAndClear(verifier: (Seq[Throwable]) => Unit): Unit = { + val faultsSoFar = faults.toSeq + try { +verifier.apply(faultsSoFar) + } catch { +case ae: AssertionError => fail(s"Assertion failed. Faults on $nodeId were: $faultsSoFar", ae) + } +} + +def waitForError(message: String): CompletableFuture[Throwable] = { + future = new CompletableFuture[Throwable]() + waitingForMsg =
(kafka) branch KAFKA-16101 updated (7690f2abed5 -> 30cbfd1609f)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git from 7690f2abed5 address review comments add 30cbfd1609f fix No new revisions were added by this update. Summary of changes: docs/ops.html | 5 - 1 file changed, 4 insertions(+), 1 deletion(-)
(kafka) branch KAFKA-16101 updated (6233d844c4a -> 7690f2abed5)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git from 6233d844c4a add table add 7690f2abed5 address review comments No new revisions were added by this update. Summary of changes: docs/ops.html | 117 +++--- 1 file changed, 88 insertions(+), 29 deletions(-)
(kafka) branch KAFKA-16101 updated (6ec8ad0ad54 -> 6233d844c4a)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git from 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect add 6233d844c4a add table No new revisions were added by this update. Summary of changes: docs/ops.html | 90 ++- 1 file changed, 70 insertions(+), 20 deletions(-)
(kafka) branch 3.7 updated: KAFKA-16078: Be more consistent about getting the latest MetadataVersion
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 3c92330274a KAFKA-16078: Be more consistent about getting the latest MetadataVersion 3c92330274a is described below commit 3c92330274a6d5ac7b2923b27f15b8d1c426da15 Author: David Arthur AuthorDate: Wed Jan 17 17:59:22 2024 -0500 KAFKA-16078: Be more consistent about getting the latest MetadataVersion This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured. Reviewers: Colin P. McCabe , Ismael Juma --- .../kafka/server/builders/LogManagerBuilder.java | 2 +- core/src/main/scala/kafka/log/UnifiedLog.scala | 4 +- .../scala/kafka/server/ApiVersionManager.scala | 2 +- .../main/scala/kafka/server/BrokerFeatures.scala | 15 ++-- .../src/main/scala/kafka/server/BrokerServer.scala | 2 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/test/java/kafka/test/ClusterConfig.java | 2 +- .../java/kafka/testkit/KafkaClusterTestKit.java| 1 + core/src/test/java/kafka/testkit/TestKitNodes.java | 2 +- .../kafka/admin/ConfigCommandIntegrationTest.scala | 2 +- .../kafka/server/KRaftClusterTest.scala| 4 +- .../MetadataRequestBetweenDifferentIbpTest.scala | 4 +- .../server/MetadataVersionIntegrationTest.scala| 4 +- .../kafka/server/QuorumTestHarness.scala | 2 +- .../unit/kafka/cluster/AbstractPartitionTest.scala | 2 +- .../unit/kafka/cluster/PartitionLockTest.scala | 2 +- .../scala/unit/kafka/cluster/PartitionTest.scala | 26 +++ .../controller/ControllerChannelManagerTest.scala | 8 +-- .../controller/ControllerIntegrationTest.scala | 4 +- .../group/GroupMetadataManagerTest.scala | 12 ++-- .../test/scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogValidatorTest.scala| 80 +++--- .../unit/kafka/network/SocketServerTest.scala | 2 +- .../kafka/security/auth/ZkAuthorizationTest.scala | 2 +- .../server/AbstractApiVersionsRequestTest.scala| 6 +- .../AlterUserScramCredentialsRequestTest.scala | 2 +- .../unit/kafka/server/ApiVersionManagerTest.scala | 4 +- .../unit/kafka/server/ApiVersionsRequestTest.scala | 16 - .../unit/kafka/server/BrokerFeaturesTest.scala | 10 +-- .../unit/kafka/server/ControllerApisTest.scala | 2 +- .../kafka/server/FinalizedFeatureCacheTest.scala | 10 +-- .../FinalizedFeatureChangeListenerTest.scala | 2 +- .../scala/unit/kafka/server/KafkaApisTest.scala| 8 +-- .../scala/unit/kafka/server/KafkaConfigTest.scala | 4 +- .../unit/kafka/server/KafkaRaftServerTest.scala| 6 +- .../scala/unit/kafka/server/KafkaServerTest.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 6 +- .../server/ReplicaAlterLogDirsThreadTest.scala | 2 +- .../kafka/server/ReplicaFetcherThreadTest.scala| 4 +- .../server/ReplicaManagerConcurrencyTest.scala | 2 +- .../unit/kafka/server/ReplicaManagerTest.scala | 2 +- ...chDrivenReplicationProtocolAcceptanceTest.scala | 2 +- .../scala/unit/kafka/tools/StorageToolTest.scala | 16 ++--- .../test/scala/unit/kafka/utils/TestUtils.scala| 4 +- .../scala/unit/kafka/zk/KafkaZkClientTest.scala| 2 +- .../group/GroupMetadataManagerTest.java| 54 +++ .../group/OffsetMetadataManagerTest.java | 2 +- .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java | 4 +- .../metadata/KRaftMetadataRequestBenchmark.java| 2 +- .../jmh/metadata/MetadataRequestBenchmark.java | 4 +- .../partition/PartitionMakeFollowerBenchmark.java | 4 +- .../UpdateFollowerFetchStateBenchmark.java | 4 +- .../CompressedRecordBatchValidationBenchmark.java | 2 +- ...UncompressedRecordBatchValidationBenchmark.java | 2 +- .../apache/kafka/jmh/server/CheckpointBench.java | 2 +- .../kafka/jmh/server/PartitionCreationBench.java | 2 +- .../kafka/controller/FeatureControlManager.java| 4 +- .../apache/kafka/controller/QuorumFeatures.java| 4 +- .../kafka/image/writer/ImageWriterOptions.java | 2 +- .../kafka/metadata/ControllerRegistration.java | 2 +- .../metadata/bootstrap/BootstrapDirectory.java | 2 +- .../controller/ClusterControlManagerTest.java | 2 +- .../controller/FeatureControlManagerTest.java | 2 +- .../controller/PartitionChangeBuilderTest.java | 2 +- .../QuorumControllerIntegrationTestUtils.java | 4 +- .../kafka/controller/QuorumControllerTestEnv.java
(kafka) branch trunk updated: KAFKA-16078: Be more consistent about getting the latest MetadataVersion
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 7bf7fd99a50 KAFKA-16078: Be more consistent about getting the latest MetadataVersion 7bf7fd99a50 is described below commit 7bf7fd99a50ba2dd71c41cd4c3623a3adce092b6 Author: David Arthur AuthorDate: Wed Jan 17 17:59:22 2024 -0500 KAFKA-16078: Be more consistent about getting the latest MetadataVersion This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured. Reviewers: Colin P. McCabe , Ismael Juma --- .../kafka/server/builders/LogManagerBuilder.java | 2 +- core/src/main/scala/kafka/log/UnifiedLog.scala | 4 +- .../scala/kafka/server/ApiVersionManager.scala | 2 +- .../main/scala/kafka/server/BrokerFeatures.scala | 15 ++-- .../src/main/scala/kafka/server/BrokerServer.scala | 2 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/test/java/kafka/test/ClusterConfig.java | 2 +- .../java/kafka/testkit/KafkaClusterTestKit.java| 1 + core/src/test/java/kafka/testkit/TestKitNodes.java | 2 +- .../kafka/admin/ConfigCommandIntegrationTest.scala | 2 +- .../kafka/server/KRaftClusterTest.scala| 4 +- .../MetadataRequestBetweenDifferentIbpTest.scala | 4 +- .../server/MetadataVersionIntegrationTest.scala| 4 +- .../kafka/server/QuorumTestHarness.scala | 2 +- .../unit/kafka/cluster/AbstractPartitionTest.scala | 2 +- .../unit/kafka/cluster/PartitionLockTest.scala | 2 +- .../scala/unit/kafka/cluster/PartitionTest.scala | 26 +++ .../controller/ControllerChannelManagerTest.scala | 8 +-- .../controller/ControllerIntegrationTest.scala | 4 +- .../group/GroupMetadataManagerTest.scala | 12 ++-- .../test/scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogValidatorTest.scala| 80 +++--- .../unit/kafka/network/SocketServerTest.scala | 2 +- .../kafka/security/auth/ZkAuthorizationTest.scala | 2 +- .../server/AbstractApiVersionsRequestTest.scala| 6 +- .../AlterUserScramCredentialsRequestTest.scala | 2 +- .../unit/kafka/server/ApiVersionManagerTest.scala | 4 +- .../unit/kafka/server/ApiVersionsRequestTest.scala | 16 - .../unit/kafka/server/BrokerFeaturesTest.scala | 10 +-- .../unit/kafka/server/ControllerApisTest.scala | 2 +- .../kafka/server/FinalizedFeatureCacheTest.scala | 10 +-- .../FinalizedFeatureChangeListenerTest.scala | 2 +- .../scala/unit/kafka/server/KafkaApisTest.scala| 8 +-- .../scala/unit/kafka/server/KafkaConfigTest.scala | 4 +- .../unit/kafka/server/KafkaRaftServerTest.scala| 6 +- .../scala/unit/kafka/server/KafkaServerTest.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 6 +- .../server/ReplicaAlterLogDirsThreadTest.scala | 2 +- .../kafka/server/ReplicaFetcherThreadTest.scala| 4 +- .../server/ReplicaManagerConcurrencyTest.scala | 2 +- ...chDrivenReplicationProtocolAcceptanceTest.scala | 2 +- .../scala/unit/kafka/tools/StorageToolTest.scala | 16 ++--- .../test/scala/unit/kafka/utils/TestUtils.scala| 4 +- .../scala/unit/kafka/zk/KafkaZkClientTest.scala| 2 +- .../group/GroupMetadataManagerTest.java| 54 +++ .../group/OffsetMetadataManagerTest.java | 2 +- .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java | 4 +- .../metadata/KRaftMetadataRequestBenchmark.java| 2 +- .../jmh/metadata/MetadataRequestBenchmark.java | 4 +- .../partition/PartitionMakeFollowerBenchmark.java | 4 +- .../UpdateFollowerFetchStateBenchmark.java | 4 +- .../CompressedRecordBatchValidationBenchmark.java | 2 +- ...UncompressedRecordBatchValidationBenchmark.java | 2 +- .../apache/kafka/jmh/server/CheckpointBench.java | 2 +- .../kafka/jmh/server/PartitionCreationBench.java | 2 +- .../kafka/controller/FeatureControlManager.java| 4 +- .../apache/kafka/controller/QuorumFeatures.java| 4 +- .../kafka/image/writer/ImageWriterOptions.java | 2 +- .../kafka/metadata/ControllerRegistration.java | 2 +- .../metadata/bootstrap/BootstrapDirectory.java | 2 +- .../controller/ClusterControlManagerTest.java | 2 +- .../controller/FeatureControlManagerTest.java | 2 +- .../controller/PartitionChangeBuilderTest.java | 2 +- .../QuorumControllerIntegrationTestUtils.java | 4 +- .../kafka/controller/QuorumControllerTestEnv.java | 2 +- .../kafka/controller/QuorumFeaturesTest.java
(kafka) branch 3.7 updated: KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 2ff3ae5bed4 KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197) 2ff3ae5bed4 is described below commit 2ff3ae5bed423f0f8212a949df28a421f97f2ab9 Author: Proven Provenzano <93720617+pprovenz...@users.noreply.github.com> AuthorDate: Wed Jan 17 17:06:38 2024 -0500 KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197) We only want to send directory assignments if the metadata version supports it. Reviewers: Colin P. McCabe --- core/src/main/scala/kafka/server/ReplicaManager.scala | 6 +- core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala | 8 +++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index fe3fad216c2..4b33a288b89 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2810,6 +2810,7 @@ class ReplicaManager(val config: KafkaConfig, def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = { // Before taking the lock, compute the local changes val localChanges = delta.localChanges(config.nodeId) +val metadataVersion = newImage.features().metadataVersion() replicaStateChangeLock.synchronized { // Handle deleted partitions. We need to do this first because we might subsequently @@ -2857,7 +2858,10 @@ class ReplicaManager(val config: KafkaConfig, remoteLogManager.foreach(rlm => rlm.onLeadershipChange(leaderChangedPartitions.asJava, followerChangedPartitions.asJava, localChanges.topicIds())) } - localChanges.directoryIds.forEach(maybeUpdateTopicAssignment) + if (metadataVersion.isDirectoryAssignmentSupported()) { +// We only want to update the directoryIds if DirectoryAssignment is supported! +localChanges.directoryIds.forEach(maybeUpdateTopicAssignment) + } } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 9201db0765e..7ef92a97371 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -57,7 +57,9 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, Node, TopicIdPartit import org.apache.kafka.image._ import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.kafka.metadata.migration.ZkMigrationState import org.apache.kafka.server.common.{DirectoryEventHandler, OffsetAndEpoch} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentMetadata, RemoteStorageException, RemoteStorageManager} import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} @@ -6171,9 +6173,13 @@ class ReplicaManagerTest { } private def imageFromTopics(topicsImage: TopicsImage): MetadataImage = { +val featuresImageLatest = new FeaturesImage( + Collections.emptyMap(), + MetadataVersion.latest(), + ZkMigrationState.NONE) new MetadataImage( new MetadataProvenance(100L, 10, 1000L), - FeaturesImage.EMPTY, + featuresImageLatest, ClusterImageTest.IMAGE1, topicsImage, ConfigurationsImage.EMPTY,
(kafka) branch trunk updated: KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 6c14d779989 KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197) 6c14d779989 is described below commit 6c14d779989dc7fec5f03c18830f1f3b80ad693e Author: Proven Provenzano <93720617+pprovenz...@users.noreply.github.com> AuthorDate: Wed Jan 17 17:06:38 2024 -0500 KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197) We only want to send directory assignments if the metadata version supports it. Reviewers: Colin P. McCabe --- core/src/main/scala/kafka/server/ReplicaManager.scala | 6 +- core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala | 8 +++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index bcc01b675ed..a987ce0f7fa 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2798,6 +2798,7 @@ class ReplicaManager(val config: KafkaConfig, def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = { // Before taking the lock, compute the local changes val localChanges = delta.localChanges(config.nodeId) +val metadataVersion = newImage.features().metadataVersion() replicaStateChangeLock.synchronized { // Handle deleted partitions. We need to do this first because we might subsequently @@ -2845,7 +2846,10 @@ class ReplicaManager(val config: KafkaConfig, remoteLogManager.foreach(rlm => rlm.onLeadershipChange(leaderChangedPartitions.asJava, followerChangedPartitions.asJava, localChanges.topicIds())) } - localChanges.directoryIds.forEach(maybeUpdateTopicAssignment) + if (metadataVersion.isDirectoryAssignmentSupported()) { +// We only want to update the directoryIds if DirectoryAssignment is supported! +localChanges.directoryIds.forEach(maybeUpdateTopicAssignment) + } } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 9201db0765e..7ef92a97371 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -57,7 +57,9 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, Node, TopicIdPartit import org.apache.kafka.image._ import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.kafka.metadata.migration.ZkMigrationState import org.apache.kafka.server.common.{DirectoryEventHandler, OffsetAndEpoch} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentMetadata, RemoteStorageException, RemoteStorageManager} import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} @@ -6171,9 +6173,13 @@ class ReplicaManagerTest { } private def imageFromTopics(topicsImage: TopicsImage): MetadataImage = { +val featuresImageLatest = new FeaturesImage( + Collections.emptyMap(), + MetadataVersion.latest(), + ZkMigrationState.NONE) new MetadataImage( new MetadataProvenance(100L, 10, 1000L), - FeaturesImage.EMPTY, + featuresImageLatest, ClusterImageTest.IMAGE1, topicsImage, ConfigurationsImage.EMPTY,
(kafka) branch 3.6 updated: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new b743f6fd884 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup b743f6fd884 is described below commit b743f6fd884132c7a5c4e9d96ed62e3aec29007f Author: Colin P. McCabe AuthorDate: Tue Jan 16 16:02:26 2024 -0800 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup Some kcontroller dynamic configurations may fail to apply at startup. This happens because there is a race between registering the reconfigurables to the DynamicBrokerConfig class, and receiving the first update from the metadata publisher. We can fix this by registering the reconfigurables first. This seems to have been introduced by the "MINOR: Install ControllerServer metadata publishers sooner" change. Reviewers: Ron Dagostino --- .../main/scala/kafka/server/ControllerServer.scala | 11 --- .../scala/kafka/server/KafkaRequestHandler.scala | 2 +- core/src/test/java/kafka/testkit/TestKitNodes.java | 26 +--- .../kafka/server/KRaftClusterTest.scala| 35 +- 4 files changed, 57 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 9a626ab84b5..5ffc382cf8d 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -330,6 +330,13 @@ class ControllerServer( ), "controller")) + // Register this instance for dynamic config changes to the KafkaConfig. This must be called + // after the authorizer and quotaManagers are initialized, since it references those objects. + // It must be called before DynamicClientQuotaPublisher is installed, since otherwise we may + // miss the initial update which establishes the dynamic configurations that are in effect on + // startup. + config.dynamicConfig.addReconfigurables(this) + // Set up the client quotas publisher. This will enable controller mutation quotas and any // other quotas which are applicable. metadataPublishers.add(new DynamicClientQuotaPublisher( @@ -346,7 +353,6 @@ class ControllerServer( credentialProvider )) - // Set up the DelegationToken publisher. // We need a tokenManager for the Publisher // The tokenCache in the tokenManager is the same used in DelegationTokenControlManager @@ -397,9 +403,6 @@ class ControllerServer( FutureUtils.waitWithLogging(logger.underlying, logIdent, "all of the SocketServer Acceptors to be started", socketServerFuture, startupDeadline, time) - - // register this instance for dynamic config changes to the KafkaConfig - config.dynamicConfig.addReconfigurables(this) } catch { case e: Throwable => maybeChangeStatus(STARTING, STARTED) diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 7885b436c59..c1c82f07e57 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -196,7 +196,7 @@ class KafkaRequestHandlerPool(val brokerId: Int, logAndThreadNamePrefix : String) extends Logging { private val metricsGroup = new KafkaMetricsGroup(this.getClass) - private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) + val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) /* a meter to track the average free capacity of the request handlers */ private val aggregateIdleMeter = metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", TimeUnit.NANOSECONDS) diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java index 5bc36854494..da4e10a4431 100644 --- a/core/src/test/java/kafka/testkit/TestKitNodes.java +++ b/core/src/test/java/kafka/testkit/TestKitNodes.java @@ -20,6 +20,7 @@ package kafka.testkit; import kafka.server.MetaProperties; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.server.common.MetadataVersion; import java.nio.file.Paths; @@ -35,7 +36,8 @@ public class TestKitNodes { public static class Builder { private boolean combined = false; private Uuid clusterId = null; -private MetadataVersion bootstrapMetadataVersion = null; +private BootstrapMetadata bootstrapMetadata = BootstrapMetadata. +fro
(kafka) branch 3.7 updated: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new b4036833081 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup b4036833081 is described below commit b40368330814888d7f7f2fda3f5b7ecfa1eabeb2 Author: Colin P. McCabe AuthorDate: Tue Jan 16 16:02:26 2024 -0800 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup Some kcontroller dynamic configurations may fail to apply at startup. This happens because there is a race between registering the reconfigurables to the DynamicBrokerConfig class, and receiving the first update from the metadata publisher. We can fix this by registering the reconfigurables first. This seems to have been introduced by the "MINOR: Install ControllerServer metadata publishers sooner" change. Reviewers: Ron Dagostino --- .../main/scala/kafka/server/ControllerServer.scala | 11 --- .../scala/kafka/server/KafkaRequestHandler.scala | 2 +- core/src/test/java/kafka/testkit/TestKitNodes.java | 25 +--- .../kafka/server/KRaftClusterTest.scala| 34 +- 4 files changed, 55 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index db386e4a92e..fb7439d8e56 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -367,6 +367,13 @@ class ControllerServer( ), "controller")) + // Register this instance for dynamic config changes to the KafkaConfig. This must be called + // after the authorizer and quotaManagers are initialized, since it references those objects. + // It must be called before DynamicClientQuotaPublisher is installed, since otherwise we may + // miss the initial update which establishes the dynamic configurations that are in effect on + // startup. + config.dynamicConfig.addReconfigurables(this) + // Set up the client quotas publisher. This will enable controller mutation quotas and any // other quotas which are applicable. metadataPublishers.add(new DynamicClientQuotaPublisher( @@ -383,7 +390,6 @@ class ControllerServer( credentialProvider )) - // Set up the DelegationToken publisher. // We need a tokenManager for the Publisher // The tokenCache in the tokenManager is the same used in DelegationTokenControlManager @@ -449,9 +455,6 @@ class ControllerServer( FutureUtils.waitWithLogging(logger.underlying, logIdent, "all of the SocketServer Acceptors to be started", socketServerFuture, startupDeadline, time) - - // register this instance for dynamic config changes to the KafkaConfig - config.dynamicConfig.addReconfigurables(this) } catch { case e: Throwable => maybeChangeStatus(STARTING, STARTED) diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 645a33ac931..895b048ca3e 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -206,7 +206,7 @@ class KafkaRequestHandlerPool( ) extends Logging { private val metricsGroup = new KafkaMetricsGroup(this.getClass) - private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) + val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) /* a meter to track the average free capacity of the request handlers */ private val aggregateIdleMeter = metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", TimeUnit.NANOSECONDS) diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java index 0f0d8939fe9..90317c56224 100644 --- a/core/src/test/java/kafka/testkit/TestKitNodes.java +++ b/core/src/test/java/kafka/testkit/TestKitNodes.java @@ -33,7 +33,8 @@ public class TestKitNodes { public static class Builder { private boolean combined = false; private Uuid clusterId = null; -private MetadataVersion bootstrapMetadataVersion = null; +private BootstrapMetadata bootstrapMetadata = BootstrapMetadata. +fromVersion(MetadataVersion.latest(), "testkit"); private final NavigableMap controllerNodeBuilders = new TreeMap<>(); private final NavigableMap brokerNodeBuilders = new TreeMap<>(); @@ -43,7 +44,12 @@ public class TestKitNodes { } public Builder setBootstrapMetadataVersion(MetadataVersion metadataVersion) { -this.bootstrapMetad
(kafka) branch trunk updated: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 0015d0f01b1 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup 0015d0f01b1 is described below commit 0015d0f01b130992acc37da85da6ee2088186a1f Author: Colin P. McCabe AuthorDate: Tue Jan 16 16:02:26 2024 -0800 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup Some kcontroller dynamic configurations may fail to apply at startup. This happens because there is a race between registering the reconfigurables to the DynamicBrokerConfig class, and receiving the first update from the metadata publisher. We can fix this by registering the reconfigurables first. This seems to have been introduced by the "MINOR: Install ControllerServer metadata publishers sooner" change. Reviewers: Ron Dagostino --- .../main/scala/kafka/server/ControllerServer.scala | 11 --- .../scala/kafka/server/KafkaRequestHandler.scala | 2 +- core/src/test/java/kafka/testkit/TestKitNodes.java | 25 +--- .../kafka/server/KRaftClusterTest.scala| 34 +- 4 files changed, 55 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index c757d6d6eea..f74bb3de7a4 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -368,6 +368,13 @@ class ControllerServer( ), "controller")) + // Register this instance for dynamic config changes to the KafkaConfig. This must be called + // after the authorizer and quotaManagers are initialized, since it references those objects. + // It must be called before DynamicClientQuotaPublisher is installed, since otherwise we may + // miss the initial update which establishes the dynamic configurations that are in effect on + // startup. + config.dynamicConfig.addReconfigurables(this) + // Set up the client quotas publisher. This will enable controller mutation quotas and any // other quotas which are applicable. metadataPublishers.add(new DynamicClientQuotaPublisher( @@ -384,7 +391,6 @@ class ControllerServer( credentialProvider )) - // Set up the DelegationToken publisher. // We need a tokenManager for the Publisher // The tokenCache in the tokenManager is the same used in DelegationTokenControlManager @@ -450,9 +456,6 @@ class ControllerServer( FutureUtils.waitWithLogging(logger.underlying, logIdent, "all of the SocketServer Acceptors to be started", socketServerFuture, startupDeadline, time) - - // register this instance for dynamic config changes to the KafkaConfig - config.dynamicConfig.addReconfigurables(this) } catch { case e: Throwable => maybeChangeStatus(STARTING, STARTED) diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 645a33ac931..895b048ca3e 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -206,7 +206,7 @@ class KafkaRequestHandlerPool( ) extends Logging { private val metricsGroup = new KafkaMetricsGroup(this.getClass) - private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) + val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) /* a meter to track the average free capacity of the request handlers */ private val aggregateIdleMeter = metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", TimeUnit.NANOSECONDS) diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java index 0f0d8939fe9..90317c56224 100644 --- a/core/src/test/java/kafka/testkit/TestKitNodes.java +++ b/core/src/test/java/kafka/testkit/TestKitNodes.java @@ -33,7 +33,8 @@ public class TestKitNodes { public static class Builder { private boolean combined = false; private Uuid clusterId = null; -private MetadataVersion bootstrapMetadataVersion = null; +private BootstrapMetadata bootstrapMetadata = BootstrapMetadata. +fromVersion(MetadataVersion.latest(), "testkit"); private final NavigableMap controllerNodeBuilders = new TreeMap<>(); private final NavigableMap brokerNodeBuilders = new TreeMap<>(); @@ -43,7 +44,12 @@ public class TestKitNodes { } public Builder setBootstrapMetadataVersion(MetadataVersion metadataVersion) { -this.bootstrapMetad
(kafka) branch 3.6 updated: KAFKA-16120: Fix partition reassignment during ZK migration
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.6 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.6 by this push: new 7486223d9e3 KAFKA-16120: Fix partition reassignment during ZK migration 7486223d9e3 is described below commit 7486223d9e39a48700bfb6dfdad93e06e4ab550f Author: David Mao AuthorDate: Fri Jan 12 09:12:33 2024 -0800 KAFKA-16120: Fix partition reassignment during ZK migration When we are migrating from ZK mode to KRaft mode, the brokers pass through a phase where they are running in ZK mode, but the controller is in KRaft mode (aka a kcontroller). This is called "hybrid mode." In hybrid mode, the KRaft controllers send old-style controller RPCs to the remaining ZK mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, UpdateMetadataRequest, etc.) To complete partition reassignment, the kcontroller must send a StopReplicaRequest to any brokers that no longer host the partition in question. Previously, it was sending this StopReplicaRequest with delete = false. This led to stray partitions, because the partition data was never removed as it should have been. This PR fixes it to set delete = true. This fixes KAFKA-16120. There is one additional problem with partition reassignment in hybrid mode, tracked as KAFKA-16121. The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where the partition leader epoch is less than or equal to the current partition leader epoch. However, when in hybrid mode, just as in KRaft mode, we do not bump the leader epoch when starting a new reassignment, see: `triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a special case on the broker side when isKRaftController = true. Reviewers: Akhilesh Chaganti , Colin P. McCabe --- .../kafka/migration/MigrationPropagator.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 84 +- 3 files changed, 85 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala b/core/src/main/scala/kafka/migration/MigrationPropagator.scala index 2a02f5891ec..7bf0fc3ff56 100644 --- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala +++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala @@ -207,7 +207,7 @@ class MigrationPropagator( val newReplicas = partitionRegistration.replicas.toSet val removedReplicas = oldReplicas -- newReplicas if (removedReplicas.nonEmpty) { - requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = false) + requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = true) } } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 469321e14d5..63a6f049247 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1818,7 +1818,7 @@ class ReplicaManager(val config: KafkaConfig, s" match the topic ID for partition $topicPartition received: " + s"${requestTopicId.get}.") responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID) - } else if (requestLeaderEpoch > currentLeaderEpoch) { + } else if (requestLeaderEpoch > currentLeaderEpoch || (requestLeaderEpoch == currentLeaderEpoch && leaderAndIsrRequest.isKRaftController)) { // If the leader epoch is valid record the epoch of the controller that made the leadership decision. // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path if (partitionState.replicas.contains(localBrokerId)) { diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index ec28484172a..4c3ddd8b80d 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -53,7 +53,7 @@ import org.slf4j.LoggerFactory import java.util import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit} -import java.util.{Properties, UUID} +import java.util.{Collections, Optional, Properties, UUID} import scala.collection.Seq import scala.jdk.CollectionConverters._ @@ -678,6 +678,88 @@ class ZkMigrationIntegrationTest { } } + @ClusterTest(clusterType = Type.ZK, brokers = 4, metadataVersion = MetadataVersion.IBP_3_6_IV2, serverProperties
(kafka) 01/01: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16126 in repository https://gitbox.apache.org/repos/asf/kafka.git commit d7f158e9d4ad3965d8e721c80c59f5e99ab9a2e7 Author: Colin P. McCabe AuthorDate: Sun Jan 14 13:50:10 2024 -0800 KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup Some kcontroller dynamic configurations may fail to apply at startup. This happens because there is a race between registering the reconfigurables to the DynamicBrokerConfig class, and receiving the first update from the metadata publisher. We can fix this by registering the reconfigurables first. This seems to have been introduced by the "MINOR: Install ControllerServer metadata publishers sooner" change. --- .../main/scala/kafka/server/ControllerServer.scala | 11 --- .../scala/kafka/server/KafkaRequestHandler.scala | 2 +- core/src/test/java/kafka/testkit/TestKitNodes.java | 25 +--- .../kafka/server/KRaftClusterTest.scala| 34 +- 4 files changed, 55 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index c757d6d6eea..f74bb3de7a4 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -368,6 +368,13 @@ class ControllerServer( ), "controller")) + // Register this instance for dynamic config changes to the KafkaConfig. This must be called + // after the authorizer and quotaManagers are initialized, since it references those objects. + // It must be called before DynamicClientQuotaPublisher is installed, since otherwise we may + // miss the initial update which establishes the dynamic configurations that are in effect on + // startup. + config.dynamicConfig.addReconfigurables(this) + // Set up the client quotas publisher. This will enable controller mutation quotas and any // other quotas which are applicable. metadataPublishers.add(new DynamicClientQuotaPublisher( @@ -384,7 +391,6 @@ class ControllerServer( credentialProvider )) - // Set up the DelegationToken publisher. // We need a tokenManager for the Publisher // The tokenCache in the tokenManager is the same used in DelegationTokenControlManager @@ -450,9 +456,6 @@ class ControllerServer( FutureUtils.waitWithLogging(logger.underlying, logIdent, "all of the SocketServer Acceptors to be started", socketServerFuture, startupDeadline, time) - - // register this instance for dynamic config changes to the KafkaConfig - config.dynamicConfig.addReconfigurables(this) } catch { case e: Throwable => maybeChangeStatus(STARTING, STARTED) diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 645a33ac931..895b048ca3e 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -206,7 +206,7 @@ class KafkaRequestHandlerPool( ) extends Logging { private val metricsGroup = new KafkaMetricsGroup(this.getClass) - private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) + val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads) /* a meter to track the average free capacity of the request handlers */ private val aggregateIdleMeter = metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", TimeUnit.NANOSECONDS) diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java index 0f0d8939fe9..90317c56224 100644 --- a/core/src/test/java/kafka/testkit/TestKitNodes.java +++ b/core/src/test/java/kafka/testkit/TestKitNodes.java @@ -33,7 +33,8 @@ public class TestKitNodes { public static class Builder { private boolean combined = false; private Uuid clusterId = null; -private MetadataVersion bootstrapMetadataVersion = null; +private BootstrapMetadata bootstrapMetadata = BootstrapMetadata. +fromVersion(MetadataVersion.latest(), "testkit"); private final NavigableMap controllerNodeBuilders = new TreeMap<>(); private final NavigableMap brokerNodeBuilders = new TreeMap<>(); @@ -43,7 +44,12 @@ public class TestKitNodes { } public Builder setBootstrapMetadataVersion(MetadataVersion metadataVersion) { -this.bootstrapMetadataVersion = metadataVersion; +this.bootstrapMetadata = BootstrapMetadata.fromVersion(metadataVersion, "testkit"); +return this; +} + +public Builder setBootstrapMetadata(BootstrapMetadata b
(kafka) branch KAFKA-16126 created (now d7f158e9d4a)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16126 in repository https://gitbox.apache.org/repos/asf/kafka.git at d7f158e9d4a KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup This branch includes the following new commits: new d7f158e9d4a KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) branch KAFKA-16101 created (now 6ec8ad0ad54)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git at 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect This branch includes the following new commits: new 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) 01/01: KAFKA-16101: KRaft migration documentation is incorrect
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16101 in repository https://gitbox.apache.org/repos/asf/kafka.git commit 6ec8ad0ad541dfe81072121d88ee75461ebeeab2 Author: Colin P. McCabe AuthorDate: Mon Jan 15 00:06:34 2024 -0800 KAFKA-16101: KRaft migration documentation is incorrect The documentation about reverting from KRaft mode to ZK mode previously said that the admin should roll each broker to remove the migration configuration, then take down the kcontrollers. However, this is not correct. Instead, the admin should roll each broker to get it into ZK mode (but not remove the migration configuration), then tke down the kcontrollers, then remove the controller znode in ZK. This PR fixes this part of the documentation. It also adds a note stating that ZK can be deprovisioned after migration is finalized. --- docs/ops.html | 16 +++- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index eb0cbd78828..62919ceab9a 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3956,18 +3956,20 @@ controller.listener.names=CONTROLLER While the cluster is still in migration mode, it is possible to revert to ZK mode. In order to do this: -For each KRaft broker: +If you have brokers in KRaft mode, for each one: Stop the broker. Remove the __cluster_metadata directory on the broker. - Remove the zookeeper.metadata.migration.enable configuration and the KRaft controllers related configurations like controller.quorum.voters -and controller.listener.names from the broker configuration properties file. - Restart the broker in ZooKeeper mode. + Reconfigure the broker so that it will restart in ZK mode, by removing process.roles + and re-adding zookeeper.connect, and so forth. + Restart the broker. - Take down the KRaft quorum. + Once all brokers are in ZK mode, take down the KRaft quorum. Using ZooKeeper shell, delete the controller node using rmr /controller, so that a ZooKeeper-based broker can become the next controller. + If desired (this step is optional) roll the brokers again, reconfiguring each without zookeeper.metadata.migration.enable. +Note that you must not remove the zookeeper.metadata.migration.enable configuration on each broker until the KRaft quorum has been completely removed. Finalizing the migration @@ -3975,6 +3977,10 @@ controller.listener.names=CONTROLLER KRaft controllers out of migration mode. This is done by removing the "zookeeper.metadata.migration.enable" property from each of their configs and restarting them one at a time. + +Once the migration has been finalized, you can safely deprovision your ZooKeeper cluster, assuming you are +not using it for anything else. After this point, it is no longer possible to revert to ZooKeeper mode. + # Sample KRaft cluster controller.properties listening on 9093
(kafka) branch 3.7 updated: KAFKA-16120: Fix partition reassignment during ZK migration
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new 4280df48161 KAFKA-16120: Fix partition reassignment during ZK migration 4280df48161 is described below commit 4280df4816195204ac2b60c7f8cdc3173bdaa375 Author: David Mao AuthorDate: Fri Jan 12 09:12:33 2024 -0800 KAFKA-16120: Fix partition reassignment during ZK migration When we are migrating from ZK mode to KRaft mode, the brokers pass through a phase where they are running in ZK mode, but the controller is in KRaft mode (aka a kcontroller). This is called "hybrid mode." In hybrid mode, the KRaft controllers send old-style controller RPCs to the remaining ZK mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, UpdateMetadataRequest, etc.) To complete partition reassignment, the kcontroller must send a StopReplicaRequest to any brokers that no longer host the partition in question. Previously, it was sending this StopReplicaRequest with delete = false. This led to stray partitions, because the partition data was never removed as it should have been. This PR fixes it to set delete = true. This fixes KAFKA-16120. There is one additional problem with partition reassignment in hybrid mode, tracked as KAFKA-16121. The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where the partition leader epoch is less than or equal to the current partition leader epoch. However, when in hybrid mode, just as in KRaft mode, we do not bump the leader epoch when starting a new reassignment, see: `triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a special case on the broker side when isKRaftController = true. Reviewers: Akhilesh Chaganti , Colin P. McCabe --- .../kafka/migration/MigrationPropagator.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 82 +- 3 files changed, 83 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala b/core/src/main/scala/kafka/migration/MigrationPropagator.scala index 2a02f5891ec..7bf0fc3ff56 100644 --- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala +++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala @@ -207,7 +207,7 @@ class MigrationPropagator( val newReplicas = partitionRegistration.replicas.toSet val removedReplicas = oldReplicas -- newReplicas if (removedReplicas.nonEmpty) { - requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = false) + requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = true) } } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b9e0e1360e9..fe3fad216c2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2072,7 +2072,7 @@ class ReplicaManager(val config: KafkaConfig, s" match the topic ID for partition $topicPartition received: " + s"${requestTopicId.get}.") responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID) - } else if (requestLeaderEpoch > currentLeaderEpoch) { + } else if (requestLeaderEpoch > currentLeaderEpoch || (requestLeaderEpoch == currentLeaderEpoch && leaderAndIsrRequest.isKRaftController)) { // If the leader epoch is valid record the epoch of the controller that made the leadership decision. // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path if (partitionState.replicas.contains(localBrokerId)) { diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 5bd1988201d..794f9b02181 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -54,7 +54,7 @@ import org.slf4j.LoggerFactory import java.util import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit} -import java.util.{Properties, UUID} +import java.util.{Collections, Optional, Properties, UUID} import scala.collection.Seq import scala.jdk.CollectionConverters._ @@ -676,6 +676,86 @@ class ZkMigrationIntegrationTest { } } + @ClusterTest(clusterType = Type.ZK, brokers = 4, metadataVersion = MetadataVersion.IBP_3_7_IV0, serverProperties
(kafka) branch trunk updated: KAFKA-16120: Fix partition reassignment during ZK migration
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new d0f845a5e1a KAFKA-16120: Fix partition reassignment during ZK migration d0f845a5e1a is described below commit d0f845a5e1a2ecceadca3978cdc6adb1eeed27e3 Author: David Mao AuthorDate: Fri Jan 12 09:12:33 2024 -0800 KAFKA-16120: Fix partition reassignment during ZK migration When we are migrating from ZK mode to KRaft mode, the brokers pass through a phase where they are running in ZK mode, but the controller is in KRaft mode (aka a kcontroller). This is called "hybrid mode." In hybrid mode, the KRaft controllers send old-style controller RPCs to the remaining ZK mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, UpdateMetadataRequest, etc.) To complete partition reassignment, the kcontroller must send a StopReplicaRequest to any brokers that no longer host the partition in question. Previously, it was sending this StopReplicaRequest with delete = false. This led to stray partitions, because the partition data was never removed as it should have been. This PR fixes it to set delete = true. This fixes KAFKA-16120. There is one additional problem with partition reassignment in hybrid mode, tracked as KAFKA-16121. The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where the partition leader epoch is less than or equal to the current partition leader epoch. However, when in hybrid mode, just as in KRaft mode, we do not bump the leader epoch when starting a new reassignment, see: `triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a special case on the broker side when isKRaftController = true. Reviewers: Akhilesh Chaganti , Colin P. McCabe --- .../kafka/migration/MigrationPropagator.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 82 +- 3 files changed, 83 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala b/core/src/main/scala/kafka/migration/MigrationPropagator.scala index 2a02f5891ec..7bf0fc3ff56 100644 --- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala +++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala @@ -207,7 +207,7 @@ class MigrationPropagator( val newReplicas = partitionRegistration.replicas.toSet val removedReplicas = oldReplicas -- newReplicas if (removedReplicas.nonEmpty) { - requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = false) + requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = true) } } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 48ec0d0fe18..bcc01b675ed 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2060,7 +2060,7 @@ class ReplicaManager(val config: KafkaConfig, s" match the topic ID for partition $topicPartition received: " + s"${requestTopicId.get}.") responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID) - } else if (requestLeaderEpoch > currentLeaderEpoch) { + } else if (requestLeaderEpoch > currentLeaderEpoch || (requestLeaderEpoch == currentLeaderEpoch && leaderAndIsrRequest.isKRaftController)) { // If the leader epoch is valid record the epoch of the controller that made the leadership decision. // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path if (partitionState.replicas.contains(localBrokerId)) { diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index d1afedd78c5..1fec4d65f09 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -55,7 +55,7 @@ import org.slf4j.LoggerFactory import java.util import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit} -import java.util.{Properties, UUID} +import java.util.{Collections, Optional, Properties, UUID} import scala.collection.Seq import scala.jdk.CollectionConverters._ @@ -677,6 +677,86 @@ class ZkMigrationIntegrationTest { } } + @ClusterTest(clusterType = Type.ZK, brokers = 4, metadataVersion = MetadataVersion.IBP_3_7_IV0, serverProper
(kafka) branch 3.7 updated: KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch 3.7 in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/3.7 by this push: new bdb4895f88f KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153) bdb4895f88f is described below commit bdb4895f88fc0753d8f2727225fd8ef72453e8c6 Author: Colin Patrick McCabe AuthorDate: Tue Jan 9 15:04:27 2024 -0800 KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153) 3.7 brokers must be able to register with 3.6 and earlier controllers. Currently, this is broken because we will unconditionally try to set logDirs, but this field cannot be sent with BrokerRegistrationRequest versions older than v2. This PR marks the logDirs field as "ignorable." Marking the field as "ignorable" means that we will still be able to send the BrokerRegistrationRequest even if the schema doesn't support logDirs. Reviewers: Ron Dagostino --- .../src/main/resources/common/message/BrokerRegistrationRequest.json | 2 +- .../java/org/apache/kafka/common/requests/RequestResponseTest.java| 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index ace268db77a..9343465ec82 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -59,7 +59,7 @@ { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", "default": "false", "about": "If the required configurations for ZK migration are present, this value is set to true" }, { "name": "LogDirs", "type": "[]uuid", "versions": "2+", - "about": "Log directories configured in this broker which are available." }, + "about": "Log directories configured in this broker which are available.", "ignorable": true }, { "name": "PreviousBrokerEpoch", "type": "int64", "versions": "3+", "default": "-1", "ignorable": true, "about": "The epoch before a clean shutdown." } ] diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b1ecb6aa4b4..4a19a3a1683 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -3502,7 +3502,9 @@ public class RequestResponseTest { new BrokerRegistrationRequestData.Feature()).iterator())) .setListeners(new BrokerRegistrationRequestData.ListenerCollection(singletonList( new BrokerRegistrationRequestData.Listener()).iterator())) -.setIncarnationId(Uuid.randomUuid()); +.setIncarnationId(Uuid.randomUuid()) + .setLogDirs(Arrays.asList(Uuid.fromString("qaJjNJ05Q36kEgeTBDcj0Q"))) +.setPreviousBrokerEpoch(123L); return new BrokerRegistrationRequest.Builder(data).build(v); }
(kafka) branch KAFKA-16094 deleted (was 584db5ede7b)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16094 in repository https://gitbox.apache.org/repos/asf/kafka.git was 584db5ede7b KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable The revisions that were on this branch are still contained in other references; therefore, this change does not discard any commits from the repository.
(kafka) branch trunk updated: KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new aa7ba7bd5af KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153) aa7ba7bd5af is described below commit aa7ba7bd5afd570c6dcf0db7d180d5a04ca084b4 Author: Colin Patrick McCabe AuthorDate: Tue Jan 9 15:04:27 2024 -0800 KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153) 3.7 brokers must be able to register with 3.6 and earlier controllers. Currently, this is broken because we will unconditionally try to set logDirs, but this field cannot be sent with BrokerRegistrationRequest versions older than v2. This PR marks the logDirs field as "ignorable." Marking the field as "ignorable" means that we will still be able to send the BrokerRegistrationRequest even if the schema doesn't support logDirs. Reviewers: Ron Dagostino --- .../src/main/resources/common/message/BrokerRegistrationRequest.json | 2 +- .../java/org/apache/kafka/common/requests/RequestResponseTest.java| 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index ace268db77a..9343465ec82 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -59,7 +59,7 @@ { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", "default": "false", "about": "If the required configurations for ZK migration are present, this value is set to true" }, { "name": "LogDirs", "type": "[]uuid", "versions": "2+", - "about": "Log directories configured in this broker which are available." }, + "about": "Log directories configured in this broker which are available.", "ignorable": true }, { "name": "PreviousBrokerEpoch", "type": "int64", "versions": "3+", "default": "-1", "ignorable": true, "about": "The epoch before a clean shutdown." } ] diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b1ecb6aa4b4..4a19a3a1683 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -3502,7 +3502,9 @@ public class RequestResponseTest { new BrokerRegistrationRequestData.Feature()).iterator())) .setListeners(new BrokerRegistrationRequestData.ListenerCollection(singletonList( new BrokerRegistrationRequestData.Listener()).iterator())) -.setIncarnationId(Uuid.randomUuid()); +.setIncarnationId(Uuid.randomUuid()) + .setLogDirs(Arrays.asList(Uuid.fromString("qaJjNJ05Q36kEgeTBDcj0Q"))) +.setPreviousBrokerEpoch(123L); return new BrokerRegistrationRequest.Builder(data).build(v); }
(kafka) branch KAFKA-16094 created (now 584db5ede7b)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch KAFKA-16094 in repository https://gitbox.apache.org/repos/asf/kafka.git at 584db5ede7b KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable This branch includes the following new commits: new 584db5ede7b KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference.
(kafka) 01/01: KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch KAFKA-16094 in repository https://gitbox.apache.org/repos/asf/kafka.git commit 584db5ede7b2d8e8345623d971a5e153eb8221b8 Author: Colin P. McCabe AuthorDate: Mon Jan 8 14:32:53 2024 -0800 KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable 3.7 brokers must be able to register with 3.6 and earlier controllers. Currently, this is broken because we will unconditionally try to set logDirs, but this field cannot be sent with BrokerRegistrationRequest versions older than v2. This PR marks the logDirs field as "ignorable." Marking the field as "ignorable" means that we will still be able to send the BrokerRegistrationRequest even if the schema doesn't support logDirs. --- .../src/main/resources/common/message/BrokerRegistrationRequest.json | 2 +- .../java/org/apache/kafka/common/requests/RequestResponseTest.java| 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index ace268db77a..9343465ec82 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -59,7 +59,7 @@ { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", "default": "false", "about": "If the required configurations for ZK migration are present, this value is set to true" }, { "name": "LogDirs", "type": "[]uuid", "versions": "2+", - "about": "Log directories configured in this broker which are available." }, + "about": "Log directories configured in this broker which are available.", "ignorable": true }, { "name": "PreviousBrokerEpoch", "type": "int64", "versions": "3+", "default": "-1", "ignorable": true, "about": "The epoch before a clean shutdown." } ] diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b1ecb6aa4b4..4a19a3a1683 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -3502,7 +3502,9 @@ public class RequestResponseTest { new BrokerRegistrationRequestData.Feature()).iterator())) .setListeners(new BrokerRegistrationRequestData.ListenerCollection(singletonList( new BrokerRegistrationRequestData.Listener()).iterator())) -.setIncarnationId(Uuid.randomUuid()); +.setIncarnationId(Uuid.randomUuid()) + .setLogDirs(Arrays.asList(Uuid.fromString("qaJjNJ05Q36kEgeTBDcj0Q"))) +.setPreviousBrokerEpoch(123L); return new BrokerRegistrationRequest.Builder(data).build(v); }
(kafka) branch trunk updated: KAFKA-15956: MetadataShell must take the log directory lock when reading (#14899)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git The following commit(s) were added to refs/heads/trunk by this push: new 32fdb8d173d KAFKA-15956: MetadataShell must take the log directory lock when reading (#14899) 32fdb8d173d is described below commit 32fdb8d173dfc1e92f4f0d3b3cf5b0b9d294b4dd Author: Colin Patrick McCabe AuthorDate: Sun Dec 10 19:18:34 2023 -0800 KAFKA-15956: MetadataShell must take the log directory lock when reading (#14899) MetadataShell should take an advisory lock on the .lock file of the directory it is reading from. Add an integration test of this functionality in MetadataShellIntegrationTest.java. Note: in build.gradle, I had to add some dependencies on server-common's test files in order to use MockFaultHandler, etc. MetadataBatchLoader.java: fix a case where a log message was incorrect. The intention was to print the number equivalent to (offset + index). Instead it was printing the offset, followed by the index. So if the offset was 100 and the index was 1, 1001 would be printed rather than 101. Co-authored-by: Igor Soarez Reviewers: David Arthur , José Armando García Sancio --- build.gradle | 5 + checkstyle/import-control.xml | 1 + .../kafka/image/loader/MetadataBatchLoader.java| 2 +- .../java/org/apache/kafka/shell/MetadataShell.java | 63 +++ .../kafka/shell/MetadataShellIntegrationTest.java | 123 + 5 files changed, 193 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index f5e8283cdc6..24b299a023d 100644 --- a/build.gradle +++ b/build.gradle @@ -2091,6 +2091,11 @@ project(':shell') { implementation libs.jacksonJaxrsJsonProvider testImplementation project(':clients') +testImplementation project(':clients').sourceSets.test.output +testImplementation project(':core') +testImplementation project(':core').sourceSets.test.output +testImplementation project(':server-common') +testImplementation project(':server-common').sourceSets.test.output testImplementation libs.junitJupiter testRuntimeOnly libs.slf4jlog4j diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 4379136bacc..1569494b455 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -275,6 +275,7 @@ + diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java index 70648bef11a..cdf95723d42 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java @@ -135,7 +135,7 @@ public class MetadataBatchLoader { replay(record); } catch (Throwable e) { faultHandler.handleFault("Error loading metadata log record from offset " + -batch.baseOffset() + indexWithinBatch, e); +(batch.baseOffset() + indexWithinBatch), e); } // Emit the accumulated delta if a new transaction has been started and one of the following is true diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java index f99695d3f12..094eabef8b3 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java @@ -19,6 +19,7 @@ package org.apache.kafka.shell; import kafka.raft.KafkaRaftManager; import kafka.tools.TerseFailure; +import kafka.utils.FileLock; import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.Namespace; @@ -36,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedWriter; +import java.io.File; import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.nio.charset.StandardCharsets; @@ -59,6 +61,8 @@ public final class MetadataShell { private String snapshotPath = null; private FaultHandler faultHandler = new LoggingFaultHandler("shell", () -> { }); +// Note: we assume that we have already taken the lock on the log directory before calling +// this method. public Builder setRaftManager(KafkaRaftManager raftManager) { this.raftManager = raftManager; return this; @@ -81,6 +85,52 @@ public final class MetadataShell { } } +/** + * Return the parent directory of a file. This works around Java's quirky API, + * which does not ho
(kafka) branch trunk updated (93b6df6173d -> 8c184b47430)
This is an automated email from the ASF dual-hosted git repository. cmccabe pushed a change to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git from 93b6df6173d KAFKA-15364: Handle log directory failure in the Controller (#14902) add 8c184b47430 MINOR: Fix some AssignmentsManager bugs (#14954) No new revisions were added by this update. Summary of changes: .../main/scala/kafka/server/ReplicaManager.scala | 3 +- .../apache/kafka/server/AssignmentsManager.java| 60 +++- .../kafka/server/AssignmentsManagerTest.java | 161 - 3 files changed, 180 insertions(+), 44 deletions(-)