From c28d9a348670f9efc599b481366da0af813aa8ae Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Wed, 8 Jan 2025 18:25:17 -0800 Subject: [PATCH] KAFKA-18435 Remove zookeeper dependencies in build.gradle (#18450) Remove Apache ZooKeeper from the Apache Kafka build. Also remove commons IO, commons CLI, and netty, which were dependencies we took only because of ZooKeeper. In order to keep the size of this PR manageable, I did not remove all classes which formerly interfaced with ZK. I just removed the ZK types. Fortunately, Kafka generally wrapped ZK data structures rather than using them directly. Some classes were pretty entangled with ZK, so it was easier just to stub them out. For ZkNodeChangeNotificationListener.scala, PartitionStateMachine.scala, ReplicaStateMachine.scala, KafkaZkClient.scala, and ZookeeperClient.scala, I replaced all the functions with "throw new UnsupportedOperationException". Since the tests for these classes have been removed, as well as the ZK-based broker code, this should be OK as an incremental step. Reviewers: Chia-Ping Tsai --- LICENSE-binary | 13 - NOTICE-binary | 39 - build.gradle | 18 - checkstyle/import-control.xml | 1 - .../ZkNodeChangeNotificationListener.scala | 110 +- .../kafka/controller/KafkaController.scala | 23 +- .../controller/PartitionStateMachine.scala | 453 +----- .../controller/ReplicaStateMachine.scala | 387 +---- .../main/scala/kafka/server/KafkaConfig.scala | 13 - .../main/scala/kafka/zk/AdminZkClient.scala | 5 - .../main/scala/kafka/zk/KafkaZkClient.scala | 1422 ++--------------- core/src/main/scala/kafka/zk/ZkData.scala | 57 +- .../kafka/zookeeper/ZooKeeperClient.scala | 379 +---- .../integration/kafka/api/SaslSetup.scala | 8 +- .../kafka/server/KRaftClusterTest.scala | 15 +- gradle/dependencies.gradle | 9 - 16 files changed, 181 insertions(+), 2771 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 5383cf60ec6..8ee1e12da86 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -208,10 +208,8 @@ License Version 2.0: audience-annotations-0.12.0 caffeine-3.1.1 commons-beanutils-1.9.4 -commons-cli-1.4 commons-collections-3.2.2 commons-digester-2.1 -commons-io-2.14.0 commons-lang3-3.12.0 commons-logging-1.3.2 commons-validator-1.9.0 @@ -257,15 +255,6 @@ lz4-java-1.8.0 maven-artifact-3.9.6 metrics-core-4.1.12.1 metrics-core-2.2.0 -netty-buffer-4.1.115.Final -netty-codec-4.1.115.Final -netty-common-4.1.115.Final -netty-handler-4.1.115.Final -netty-resolver-4.1.115.Final -netty-transport-4.1.115.Final -netty-transport-classes-epoll-4.1.115.Final -netty-transport-native-epoll-4.1.115.Final -netty-transport-native-unix-common-4.1.115.Final opentelemetry-proto-1.0.0-alpha plexus-utils-3.5.1 rocksdbjni-7.9.2 @@ -275,8 +264,6 @@ scala-reflect-2.13.15 snappy-java-1.1.10.5 snakeyaml-2.2 swagger-annotations-2.2.25 -zookeeper-3.8.4 -zookeeper-jute-3.8.4 =============================================================================== This product bundles various third-party components under other open source diff --git a/NOTICE-binary b/NOTICE-binary index 6a30e89baa7..50dabb33d61 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -596,45 +596,6 @@ limitations under the License. This software includes projects with other licenses -- see `doc/LICENSE.md`. -Apache ZooKeeper - Server -Copyright 2008-2021 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - - -Apache ZooKeeper - Jute -Copyright 2008-2021 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - - -The Netty Project - ================= - -Please visit the Netty web site for more information: - - * https://netty.io/ - -Copyright 2014 The Netty Project - -The Netty Project 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: - - https://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. - -Also, please refer to each LICENSE..txt file, which is located in -the 'license' directory of the distribution file, for the license terms of the -components that this product depends on. - ------------------------------------------------------------------------------- This product contains the extensions to Java Collections Framework which has been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: diff --git a/build.gradle b/build.gradle index 5e15200f147..61338a1bf95 100644 --- a/build.gradle +++ b/build.gradle @@ -190,10 +190,6 @@ allprojects { libs.scalaReflect, libs.jacksonAnnotations, libs.jacksonDatabindYaml, - // be explicit about the Netty dependency version instead of relying on the version set by - // ZooKeeper (potentially older and containing CVEs) - libs.nettyHandler, - libs.nettyTransportNativeEpoll, libs.log4j2Api, libs.log4j2Core, libs.log4j1Bridge2Api @@ -1107,20 +1103,6 @@ project(':core') { // only needed transitively, but set it explicitly to ensure it has the same version as scala-library implementation libs.scalaReflect implementation libs.scalaLogging - implementation libs.commonsIo // ZooKeeper dependency. Do not use, this is going away. - implementation(libs.zookeeper) { - // Dropwizard Metrics are required by ZooKeeper as of v3.6.0, - // but the library should *not* be used in Kafka code - implementation libs.dropwizardMetrics - exclude module: 'slf4j-log4j12' - exclude module: 'log4j' - // Both Kafka and Zookeeper use slf4j. ZooKeeper moved from log4j to logback in v3.8.0. - // We are removing Zookeeper's dependency on logback so we have a singular logging backend. - exclude module: 'logback-classic' - exclude module: 'logback-core' - } - // ZooKeeperMain depends on commons-cli but declares the dependency as `provided` - implementation libs.commonsCli implementation log4jLibs runtimeOnly log4jRuntimeLibs diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 88cbee167c2..921db8162e3 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -451,7 +451,6 @@ - diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala index 1f7da18d9ac..de0dd447420 100644 --- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala +++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala @@ -16,18 +16,9 @@ */ package kafka.common -import java.nio.charset.StandardCharsets.UTF_8 -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.atomic.AtomicBoolean - import kafka.utils.Logging -import kafka.zk.{KafkaZkClient, StateChangeHandlers} -import kafka.zookeeper.{StateChangeHandler, ZNodeChildChangeHandler} +import kafka.zk.KafkaZkClient import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.util.ShutdownableThread - -import scala.collection.Seq -import scala.util.{Failure, Try} /** * Handle the notificationMessage. @@ -56,105 +47,20 @@ class ZkNodeChangeNotificationListener(private val zkClient: KafkaZkClient, private val notificationHandler: NotificationHandler, private val changeExpirationMs: Long = 15 * 60 * 1000, private val time: Time = Time.SYSTEM) extends Logging { - private var lastExecutedChange = -1L - private val queue = new LinkedBlockingQueue[ChangeNotification] - private val thread = new ChangeEventProcessThread(s"$seqNodeRoot-event-process-thread") - private val isClosed = new AtomicBoolean(false) - def init(): Unit = { - zkClient.registerStateChangeHandler(ZkStateChangeHandler) - zkClient.registerZNodeChildChangeHandler(ChangeNotificationHandler) - addChangeNotification() - thread.start() + throw new UnsupportedOperationException() } def close(): Unit = { - isClosed.set(true) - zkClient.unregisterStateChangeHandler(ZkStateChangeHandler.name) - zkClient.unregisterZNodeChildChangeHandler(ChangeNotificationHandler.path) - queue.clear() - thread.shutdown() + throw new UnsupportedOperationException() } - /** - * Process notifications - */ - private def processNotifications(): Unit = { - try { - val notifications = zkClient.getChildren(seqNodeRoot).sorted - if (notifications.nonEmpty) { - info(s"Processing notification(s) to $seqNodeRoot") - val now = time.milliseconds - for (notification <- notifications) { - val changeId = changeNumber(notification) - if (changeId > lastExecutedChange) { - processNotification(notification) - lastExecutedChange = changeId - } - } - purgeObsoleteNotifications(now, notifications) - } - } catch { - case e: InterruptedException => if (!isClosed.get) error(s"Error while processing notification change for path = $seqNodeRoot", e) - case e: Exception => error(s"Error while processing notification change for path = $seqNodeRoot", e) + object ZkStateChangeHandler { + val name: String = null + + def afterInitializingSession(): Unit = { + throw new UnsupportedOperationException() } } - - private def processNotification(notification: String): Unit = { - val changeZnode = seqNodeRoot + "/" + notification - val (data, _) = zkClient.getDataAndStat(changeZnode) - data match { - case Some(d) => Try(notificationHandler.processNotification(d)) match { - case Failure(e) => error(s"error processing change notification ${new String(d, UTF_8)} from $changeZnode", e) - case _ => - } - case None => warn(s"read null data from $changeZnode") - } - } - - private def addChangeNotification(): Unit = { - if (!isClosed.get && queue.peek() == null) - queue.put(new ChangeNotification) - } - - private class ChangeNotification { - def process(): Unit = processNotifications() - } - - /** - * Purges expired notifications. - * - * @param now - * @param notifications - */ - private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]): Unit = { - for (notification <- notifications.sorted) { - val notificationNode = seqNodeRoot + "/" + notification - val (data, stat) = zkClient.getDataAndStat(notificationNode) - if (data.isDefined) { - if (now - stat.getCtime > changeExpirationMs) { - debug(s"Purging change notification $notificationNode") - zkClient.deletePath(notificationNode) - } - } - } - } - - /* get the change number from a change notification znode */ - private def changeNumber(name: String): Long = name.substring(seqNodePrefix.length).toLong - - private class ChangeEventProcessThread(name: String) extends ShutdownableThread(name) { - override def doWork(): Unit = queue.take().process() - } - - private object ChangeNotificationHandler extends ZNodeChildChangeHandler { - override val path: String = seqNodeRoot - override def handleChildChange(): Unit = addChangeNotification() - } - - object ZkStateChangeHandler extends StateChangeHandler { - override val name: String = StateChangeHandlers.zkNodeChangeListenerHandler(seqNodeRoot) - override def afterInitializingSession(): Unit = addChangeNotification() - } } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 60f05a05f31..1cc4b898c77 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -28,10 +28,9 @@ import kafka.utils._ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zk.TopicZNode.TopicIdReplicaAssignment import kafka.zk.{FeatureZNodeStatus, _} -import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler} +import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler, ZooKeeperClientException} import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType import org.apache.kafka.common.ElectionType -import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} @@ -45,8 +44,6 @@ import org.apache.kafka.server.BrokerFeatures import org.apache.kafka.server.common.{AdminOperationException, ProducerIdsBlock} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.KafkaScheduler -import org.apache.zookeeper.KeeperException -import org.apache.zookeeper.KeeperException.Code import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.collection.mutable.ArrayBuffer @@ -1068,21 +1065,7 @@ class KafkaController(val config: KafkaConfig, } private def updateReplicaAssignmentForPartition(topicPartition: TopicPartition, assignment: ReplicaAssignment): Unit = { - val topicAssignment = mutable.Map() ++= - controllerContext.partitionFullReplicaAssignmentForTopic(topicPartition.topic) += - (topicPartition -> assignment) - - val setDataResponse = zkClient.setTopicAssignmentRaw(topicPartition.topic, - controllerContext.topicIds.get(topicPartition.topic), - topicAssignment, controllerContext.epochZkVersion) - setDataResponse.resultCode match { - case Code.OK => - info(s"Successfully updated assignment of partition $topicPartition to $assignment") - case Code.NONODE => - throw new IllegalStateException(s"Failed to update assignment for $topicPartition since the topic " + - "has no current assignment") - case _ => throw new KafkaException(setDataResponse.resultException.get) - } + throw new UnsupportedOperationException() } private def startNewReplicasForReassignedPartition(topicPartition: TopicPartition, newReplicas: Seq[Int]): Unit = { @@ -1186,7 +1169,7 @@ class KafkaController(val config: KafkaConfig, try { zkClient.setOrCreatePartitionReassignment(updatedPartitionsBeingReassigned, controllerContext.epochZkVersion) } catch { - case e: KeeperException => throw new AdminOperationException(e) + case e: ZooKeeperClientException => throw new AdminOperationException(e) } } } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index c0b92b9c638..d03ac18a240 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -16,40 +16,27 @@ */ package kafka.controller -import kafka.common.StateChangeFailedException -import kafka.controller.Election._ import kafka.server.KafkaConfig import kafka.utils.Logging import kafka.zk.KafkaZkClient -import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult -import kafka.zk.TopicPartitionStateZNode import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.metadata.LeaderAndIsr -import org.apache.kafka.server.common.MetadataVersion.IBP_3_2_IV0 -import org.apache.zookeeper.KeeperException -import org.apache.zookeeper.KeeperException.Code -import scala.collection.{Map, Seq, mutable} -import scala.jdk.CollectionConverters._ +import scala.collection.{Map, Seq} abstract class PartitionStateMachine(controllerContext: ControllerContext) extends Logging { /** * Invoked on successful controller election. */ def startup(): Unit = { - info("Initializing partition state") - initializePartitionState() - info("Triggering online partition state changes") - triggerOnlinePartitionStateChange() - debug(s"Started partition state machine with initial state -> ${controllerContext.partitionStates}") + throw new UnsupportedOperationException() } /** * Invoked on controller shutdown. */ def shutdown(): Unit = { - info("Stopped partition state machine") + throw new UnsupportedOperationException() } /** @@ -57,53 +44,18 @@ abstract class PartitionStateMachine(controllerContext: ControllerContext) exten * state. This is called on a successful controller election and on broker changes */ def triggerOnlinePartitionStateChange(): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - val partitions = controllerContext.partitionsInStates(Set(OfflinePartition, NewPartition)) - triggerOnlineStateChangeForPartitions(partitions) + throw new UnsupportedOperationException() } def triggerOnlinePartitionStateChange(topic: String): Unit = { - val partitions = controllerContext.partitionsInStates(topic, Set(OfflinePartition, NewPartition)) - triggerOnlineStateChangeForPartitions(partitions) - } - - private def triggerOnlineStateChangeForPartitions(partitions: collection.Set[TopicPartition]): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - // try to move all partitions in NewPartition or OfflinePartition state to OnlinePartition state except partitions - // that belong to topics to be deleted - val partitionsToTrigger = partitions.filter { partition => - !controllerContext.isTopicQueuedUpForDeletion(partition.topic) - }.toSeq - - handleStateChanges(partitionsToTrigger, OnlinePartition, Some(OfflinePartitionLeaderElectionStrategy(false))) - // TODO: If handleStateChanges catches an exception, it is not enough to bail out and log an error. - // It is important to trigger leader election for those partitions. - } - - /** - * Invoked on startup of the partition's state machine to set the initial state for all existing partitions in - * zookeeper - */ - private def initializePartitionState(): Unit = { - for (topicPartition <- controllerContext.allPartitions) { - // check if leader and isr path exists for partition. If not, then it is in NEW state - controllerContext.partitionLeadershipInfo(topicPartition) match { - case Some(currentLeaderIsrAndEpoch) => - // else, check if the leader for partition is alive. If yes, it is in Online state, else it is in Offline state - if (controllerContext.isReplicaOnline(currentLeaderIsrAndEpoch.leaderAndIsr.leader, topicPartition)) - // leader is alive - controllerContext.putPartitionState(topicPartition, OnlinePartition) - else - controllerContext.putPartitionState(topicPartition, OfflinePartition) - case None => - controllerContext.putPartitionState(topicPartition, NewPartition) - } - } + throw new UnsupportedOperationException() } def handleStateChanges( partitions: Seq[TopicPartition], targetState: PartitionState ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - handleStateChanges(partitions, targetState, None) + throw new UnsupportedOperationException() } def handleStateChanges( @@ -111,7 +63,6 @@ abstract class PartitionStateMachine(controllerContext: ControllerContext) exten targetState: PartitionState, leaderElectionStrategy: Option[PartitionLeaderElectionStrategy] ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] - } /** @@ -133,8 +84,6 @@ class ZkPartitionStateMachine(config: KafkaConfig, controllerBrokerRequestBatch: ControllerBrokerRequestBatch) extends PartitionStateMachine(controllerContext) { - private val isLeaderRecoverySupported = config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0) - private val controllerId = config.brokerId this.logIdent = s"[PartitionStateMachine controllerId=$controllerId] " @@ -153,405 +102,25 @@ class ZkPartitionStateMachine(config: KafkaConfig, targetState: PartitionState, partitionLeaderElectionStrategyOpt: Option[PartitionLeaderElectionStrategy] ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - if (partitions.nonEmpty) { - try { - controllerBrokerRequestBatch.newBatch() - val result = doHandleStateChanges( - partitions, - targetState, - partitionLeaderElectionStrategyOpt - ) - controllerBrokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch) - result - } catch { - case e: ControllerMovedException => - error(s"Controller moved to another broker when moving some partitions to $targetState state", e) - throw e - case e: Throwable => - error(s"Error while moving some partitions to $targetState state", e) - partitions.iterator.map(_ -> Left(e)).toMap - } - } else { - Map.empty - } - } - - private def partitionState(partition: TopicPartition): PartitionState = { - controllerContext.partitionState(partition) - } - - /** - * This API exercises the partition's state machine. It ensures that every state transition happens from a legal - * previous state to the target state. Valid state transitions are: - * NonExistentPartition -> NewPartition: - * --load assigned replicas from ZK to controller cache - * - * NewPartition -> OnlinePartition - * --assign first live replica as the leader and all live replicas as the isr; write leader and isr to ZK for this partition - * --send LeaderAndIsr request to every live replica and UpdateMetadata request to every live broker - * - * OnlinePartition,OfflinePartition -> OnlinePartition - * --select new leader and isr for this partition and a set of replicas to receive the LeaderAndIsr request, and write leader and isr to ZK - * --for this partition, send LeaderAndIsr request to every receiving replica and UpdateMetadata request to every live broker - * - * NewPartition,OnlinePartition,OfflinePartition -> OfflinePartition - * --nothing other than marking partition state as Offline - * - * OfflinePartition -> NonExistentPartition - * --nothing other than marking the partition state as NonExistentPartition - * @param partitions The partitions for which the state transition is invoked - * @param targetState The end state that the partition should be moved to - * @return A map of failed and successful elections when targetState is OnlinePartitions. The keys are the - * topic partitions and the corresponding values are either the exception that was thrown or new - * leader & ISR. - */ - private def doHandleStateChanges( - partitions: Seq[TopicPartition], - targetState: PartitionState, - partitionLeaderElectionStrategyOpt: Option[PartitionLeaderElectionStrategy] - ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - val stateChangeLog = stateChangeLogger.withControllerEpoch(controllerContext.epoch) - val traceEnabled = stateChangeLog.isTraceEnabled - partitions.foreach(partition => controllerContext.putPartitionStateIfNotExists(partition, NonExistentPartition)) - val (validPartitions, invalidPartitions) = controllerContext.checkValidPartitionStateChange(partitions, targetState) - invalidPartitions.foreach(partition => logInvalidTransition(partition, targetState)) - - targetState match { - case NewPartition => - validPartitions.foreach { partition => - stateChangeLog.info(s"Changed partition $partition state from ${partitionState(partition)} to $targetState with " + - s"assigned replicas ${controllerContext.partitionReplicaAssignment(partition).mkString(",")}") - controllerContext.putPartitionState(partition, NewPartition) - } - Map.empty - case OnlinePartition => - val uninitializedPartitions = validPartitions.filter(partition => partitionState(partition) == NewPartition) - val partitionsToElectLeader = validPartitions.filter(partition => partitionState(partition) == OfflinePartition || partitionState(partition) == OnlinePartition) - if (uninitializedPartitions.nonEmpty) { - val successfulInitializations = initializeLeaderAndIsrForPartitions(uninitializedPartitions) - successfulInitializations.foreach { partition => - stateChangeLog.info(s"Changed partition $partition from ${partitionState(partition)} to $targetState with state " + - s"${controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr}") - controllerContext.putPartitionState(partition, OnlinePartition) - } - } - if (partitionsToElectLeader.nonEmpty) { - val electionResults = electLeaderForPartitions( - partitionsToElectLeader, - partitionLeaderElectionStrategyOpt.getOrElse( - throw new IllegalArgumentException("Election strategy is a required field when the target state is OnlinePartition") - ) - ) - - electionResults.foreach { - case (partition, Right(leaderAndIsr)) => - stateChangeLog.info( - s"Changed partition $partition from ${partitionState(partition)} to $targetState with state $leaderAndIsr" - ) - controllerContext.putPartitionState(partition, OnlinePartition) - case (_, Left(_)) => // Ignore; no need to update partition state on election error - } - - electionResults - } else { - Map.empty - } - case OfflinePartition | NonExistentPartition => - validPartitions.foreach { partition => - if (traceEnabled) - stateChangeLog.trace(s"Changed partition $partition state from ${partitionState(partition)} to $targetState") - controllerContext.putPartitionState(partition, targetState) - } - Map.empty - } - } - - /** - * Initialize leader and isr partition state in zookeeper. - * @param partitions The partitions that we're trying to initialize. - * @return The partitions that have been successfully initialized. - */ - private def initializeLeaderAndIsrForPartitions(partitions: Seq[TopicPartition]): Seq[TopicPartition] = { - val successfulInitializations = mutable.Buffer.empty[TopicPartition] - val replicasPerPartition = partitions.map(partition => partition -> controllerContext.partitionReplicaAssignment(partition)) - val liveReplicasPerPartition = replicasPerPartition.map { case (partition, replicas) => - val liveReplicasForPartition = replicas.filter(replica => controllerContext.isReplicaOnline(replica, partition)) - partition -> liveReplicasForPartition - } - val (partitionsWithoutLiveReplicas, partitionsWithLiveReplicas) = liveReplicasPerPartition.partition { case (_, liveReplicas) => liveReplicas.isEmpty } - - partitionsWithoutLiveReplicas.foreach { case (partition, _) => - val failMsg = s"Controller $controllerId epoch ${controllerContext.epoch} encountered error during state change of " + - s"partition $partition from New to Online, assigned replicas are " + - s"[${controllerContext.partitionReplicaAssignment(partition).mkString(",")}], live brokers are [${controllerContext.liveBrokerIds}]. No assigned " + - "replica is alive." - logFailedStateChange(partition, NewPartition, OnlinePartition, new StateChangeFailedException(failMsg)) - } - val leaderIsrAndControllerEpochs = partitionsWithLiveReplicas.map { case (partition, liveReplicas) => - val leaderAndIsr = new LeaderAndIsr(liveReplicas.head, liveReplicas.toList.map(Integer.valueOf).asJava) - val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) - partition -> leaderIsrAndControllerEpoch - }.toMap - val createResponses = try { - zkClient.createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs, controllerContext.epochZkVersion) - } catch { - case e: ControllerMovedException => - error("Controller moved to another broker when trying to create the topic partition state znode", e) - throw e - case e: Exception => - partitionsWithLiveReplicas.foreach { case (partition, _) => logFailedStateChange(partition, partitionState(partition), NewPartition, e) } - Seq.empty - } - createResponses.foreach { createResponse => - val code = createResponse.resultCode - val partition = createResponse.ctx.get.asInstanceOf[TopicPartition] - val leaderIsrAndControllerEpoch = leaderIsrAndControllerEpochs(partition) - if (code == Code.OK) { - controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) - controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(leaderIsrAndControllerEpoch.leaderAndIsr.isr.asScala.map(_.toInt), - partition, leaderIsrAndControllerEpoch, controllerContext.partitionFullReplicaAssignment(partition), isNew = true) - successfulInitializations += partition - } else { - logFailedStateChange(partition, NewPartition, OnlinePartition, code) - } - } - successfulInitializations - } - - /** - * Repeatedly attempt to elect leaders for multiple partitions until there are no more remaining partitions to retry. - * @param partitions The partitions that we're trying to elect leaders for. - * @param partitionLeaderElectionStrategy The election strategy to use. - * @return A map of failed and successful elections. The keys are the topic partitions and the corresponding values are - * either the exception that was thrown or new leader & ISR. - */ - private def electLeaderForPartitions( - partitions: Seq[TopicPartition], - partitionLeaderElectionStrategy: PartitionLeaderElectionStrategy - ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - var remaining = partitions - val finishedElections = mutable.Map.empty[TopicPartition, Either[Throwable, LeaderAndIsr]] - - while (remaining.nonEmpty) { - val (finished, updatesToRetry) = doElectLeaderForPartitions(remaining, partitionLeaderElectionStrategy) - remaining = updatesToRetry - - finished.foreach { - case (partition, Left(e)) => - logFailedStateChange(partition, partitionState(partition), OnlinePartition, e) - case (_, Right(_)) => // Ignore; success so no need to log failed state change - } - - finishedElections ++= finished - - if (remaining.nonEmpty) - logger.info(s"Retrying leader election with strategy $partitionLeaderElectionStrategy for partitions $remaining") - } - - finishedElections.toMap - } - - /** - * Try to elect leaders for multiple partitions. - * Electing a leader for a partition updates partition state in zookeeper. - * - * @param partitions The partitions that we're trying to elect leaders for. - * @param partitionLeaderElectionStrategy The election strategy to use. - * @return A tuple of two values: - * 1. The partitions and the expected leader and isr that successfully had a leader elected. And exceptions - * corresponding to failed elections that should not be retried. - * 2. The partitions that we should retry due to a zookeeper BADVERSION conflict. Version conflicts can occur if - * the partition leader updated partition state while the controller attempted to update partition state. - */ - private def doElectLeaderForPartitions( - partitions: Seq[TopicPartition], - partitionLeaderElectionStrategy: PartitionLeaderElectionStrategy - ): (Map[TopicPartition, Either[Exception, LeaderAndIsr]], Seq[TopicPartition]) = { - val getDataResponses = try { - zkClient.getTopicPartitionStatesRaw(partitions) - } catch { - case e: Exception => - return (partitions.iterator.map(_ -> Left(e)).toMap, Seq.empty) - } - val failedElections = mutable.Map.empty[TopicPartition, Either[Exception, LeaderAndIsr]] - val validLeaderAndIsrs = mutable.Buffer.empty[(TopicPartition, LeaderAndIsr)] - - getDataResponses.foreach { getDataResponse => - val partition = getDataResponse.ctx.get.asInstanceOf[TopicPartition] - val currState = partitionState(partition) - if (getDataResponse.resultCode == Code.OK) { - TopicPartitionStateZNode.decode(getDataResponse.data, getDataResponse.stat) match { - case Some(leaderIsrAndControllerEpoch) => - if (leaderIsrAndControllerEpoch.controllerEpoch > controllerContext.epoch) { - val failMsg = s"Aborted leader election for partition $partition since the LeaderAndIsr path was " + - s"already written by another controller. This probably means that the current controller $controllerId went through " + - s"a soft failure and another controller was elected with epoch ${leaderIsrAndControllerEpoch.controllerEpoch}." - failedElections.put(partition, Left(new StateChangeFailedException(failMsg))) - } else { - validLeaderAndIsrs += partition -> leaderIsrAndControllerEpoch.leaderAndIsr - } - - case None => - val exception = new StateChangeFailedException(s"LeaderAndIsr information doesn't exist for partition $partition in $currState state") - failedElections.put(partition, Left(exception)) - } - - } else if (getDataResponse.resultCode == Code.NONODE) { - val exception = new StateChangeFailedException(s"LeaderAndIsr information doesn't exist for partition $partition in $currState state") - failedElections.put(partition, Left(exception)) - } else { - failedElections.put(partition, Left(getDataResponse.resultException.get)) - } - } - - if (validLeaderAndIsrs.isEmpty) { - return (failedElections.toMap, Seq.empty) - } - - val (partitionsWithoutLeaders, partitionsWithLeaders) = partitionLeaderElectionStrategy match { - case OfflinePartitionLeaderElectionStrategy(allowUnclean) => - val partitionsWithUncleanLeaderElectionState = collectUncleanLeaderElectionState( - validLeaderAndIsrs, - allowUnclean - ) - leaderForOffline( - controllerContext, - isLeaderRecoverySupported, - partitionsWithUncleanLeaderElectionState - ).partition(_.leaderAndIsr.isEmpty) - - case ReassignPartitionLeaderElectionStrategy => - leaderForReassign(controllerContext, validLeaderAndIsrs).partition(_.leaderAndIsr.isEmpty) - case PreferredReplicaPartitionLeaderElectionStrategy => - leaderForPreferredReplica(controllerContext, validLeaderAndIsrs).partition(_.leaderAndIsr.isEmpty) - case ControlledShutdownPartitionLeaderElectionStrategy => - leaderForControlledShutdown(controllerContext, validLeaderAndIsrs).partition(_.leaderAndIsr.isEmpty) - } - partitionsWithoutLeaders.foreach { electionResult => - val partition = electionResult.topicPartition - val failMsg = s"Failed to elect leader for partition $partition under strategy $partitionLeaderElectionStrategy" - failedElections.put(partition, Left(new StateChangeFailedException(failMsg))) - } - val recipientsPerPartition = partitionsWithLeaders.map(result => result.topicPartition -> result.liveReplicas).toMap - val adjustedLeaderAndIsrs = partitionsWithLeaders.map(result => result.topicPartition -> result.leaderAndIsr.get).toMap - val UpdateLeaderAndIsrResult(finishedUpdates, updatesToRetry) = zkClient.updateLeaderAndIsr( - adjustedLeaderAndIsrs, controllerContext.epoch, controllerContext.epochZkVersion) - finishedUpdates.foreachEntry { (partition, result) => - result.foreach { leaderAndIsr => - val replicaAssignment = controllerContext.partitionFullReplicaAssignment(partition) - val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) - controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) - controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(recipientsPerPartition(partition), partition, - leaderIsrAndControllerEpoch, replicaAssignment, isNew = false) - } - } - - if (isDebugEnabled) { - updatesToRetry.foreach { partition => - debug(s"Controller failed to elect leader for partition $partition. " + - s"Attempted to write state ${adjustedLeaderAndIsrs(partition)}, but failed with bad ZK version. This will be retried.") - } - } - - (finishedUpdates ++ failedElections, updatesToRetry) - } - - /* For the provided set of topic partition and partition sync state it attempts to determine if unclean - * leader election should be performed. Unclean election should be performed if there are no live - * replica which are in sync and unclean leader election is allowed (allowUnclean parameter is true or - * the topic has been configured to allow unclean election). - * - * @param leaderIsrAndControllerEpochs set of partition to determine if unclean leader election should be - * allowed - * @param allowUnclean whether to allow unclean election without having to read the topic configuration - * @return a sequence of three element tuple: - * 1. topic partition - * 2. leader, isr and controller epoc. Some means election should be performed - * 3. allow unclean - */ - private def collectUncleanLeaderElectionState( - leaderAndIsrs: Seq[(TopicPartition, LeaderAndIsr)], - allowUnclean: Boolean - ): Seq[(TopicPartition, Option[LeaderAndIsr], Boolean)] = { - val (partitionsWithNoLiveInSyncReplicas, partitionsWithLiveInSyncReplicas) = leaderAndIsrs.partition { - case (partition, leaderAndIsr) => - val liveInSyncReplicas = leaderAndIsr.isr.asScala.filter(controllerContext.isReplicaOnline(_, partition)) - liveInSyncReplicas.isEmpty - } - - val electionForPartitionWithoutLiveReplicas = if (allowUnclean) { - partitionsWithNoLiveInSyncReplicas.map { case (partition, leaderAndIsr) => - (partition, Option(leaderAndIsr), true) - } - } else { - val (logConfigs, failed) = zkClient.getLogConfigs( - partitionsWithNoLiveInSyncReplicas.iterator.map { case (partition, _) => partition.topic }.toSet, - config.originals() - ) - - partitionsWithNoLiveInSyncReplicas.map { case (partition, leaderAndIsr) => - if (failed.contains(partition.topic)) { - logFailedStateChange(partition, partitionState(partition), OnlinePartition, failed(partition.topic)) - (partition, None, false) - } else { - ( - partition, - Option(leaderAndIsr), - logConfigs(partition.topic).uncleanLeaderElectionEnable.booleanValue() - ) - } - } - } - - electionForPartitionWithoutLiveReplicas ++ - partitionsWithLiveInSyncReplicas.map { case (partition, leaderAndIsr) => - (partition, Option(leaderAndIsr), false) - } - } - - private def logInvalidTransition(partition: TopicPartition, targetState: PartitionState): Unit = { - val currState = partitionState(partition) - val e = new IllegalStateException(s"Partition $partition should be in one of " + - s"${targetState.validPreviousStates.mkString(",")} states before moving to $targetState state. Instead it is in " + - s"$currState state") - logFailedStateChange(partition, currState, targetState, e) - } - - private def logFailedStateChange(partition: TopicPartition, currState: PartitionState, targetState: PartitionState, code: Code): Unit = { - logFailedStateChange(partition, currState, targetState, KeeperException.create(code)) - } - - private def logFailedStateChange(partition: TopicPartition, currState: PartitionState, targetState: PartitionState, t: Throwable): Unit = { - stateChangeLogger.withControllerEpoch(controllerContext.epoch) - .error(s"Controller $controllerId epoch ${controllerContext.epoch} failed to change state for partition $partition " + - s"from $currState to $targetState", t) + throw new UnsupportedOperationException() } } object PartitionLeaderElectionAlgorithms { def offlinePartitionLeaderElection(assignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int], uncleanLeaderElectionEnabled: Boolean, controllerContext: ControllerContext): Option[Int] = { - assignment.find(id => liveReplicas.contains(id) && isr.contains(id)).orElse { - if (uncleanLeaderElectionEnabled) { - val leaderOpt = assignment.find(liveReplicas.contains) - if (leaderOpt.isDefined) - controllerContext.stats.uncleanLeaderElectionRate.mark() - leaderOpt - } else { - None - } - } + throw new UnsupportedOperationException() } def reassignPartitionLeaderElection(reassignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int]): Option[Int] = { - reassignment.find(id => liveReplicas.contains(id) && isr.contains(id)) + throw new UnsupportedOperationException() } def preferredReplicaPartitionLeaderElection(assignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int]): Option[Int] = { - assignment.headOption.filter(id => liveReplicas.contains(id) && isr.contains(id)) + throw new UnsupportedOperationException() } def controlledShutdownPartitionLeaderElection(assignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int], shuttingDownBrokers: Set[Int]): Option[Int] = { - assignment.find(id => liveReplicas.contains(id) && isr.contains(id) && !shuttingDownBrokers.contains(id)) + throw new UnsupportedOperationException() } } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 406fff2b51b..7a58f277d0f 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -16,61 +16,25 @@ */ package kafka.controller -import kafka.common.StateChangeFailedException import kafka.server.KafkaConfig import kafka.utils.Logging import kafka.zk.KafkaZkClient -import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult -import kafka.zk.TopicPartitionStateZNode -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.ControllerMovedException -import org.apache.kafka.metadata.LeaderAndIsr -import org.apache.zookeeper.KeeperException.Code -import java.util.stream.Collectors -import scala.collection.{Seq, mutable} +import scala.collection.Seq abstract class ReplicaStateMachine(controllerContext: ControllerContext) extends Logging { /** * Invoked on successful controller election. */ def startup(): Unit = { - info("Initializing replica state") - initializeReplicaState() - info("Triggering online replica state changes") - val (onlineReplicas, offlineReplicas) = controllerContext.onlineAndOfflineReplicas - handleStateChanges(onlineReplicas.toSeq, OnlineReplica) - info("Triggering offline replica state changes") - handleStateChanges(offlineReplicas.toSeq, OfflineReplica) - debug(s"Started replica state machine with initial state -> ${controllerContext.replicaStates}") + throw new UnsupportedOperationException() } /** * Invoked on controller shutdown. */ def shutdown(): Unit = { - info("Stopped replica state machine") - } - - /** - * Invoked on startup of the replica's state machine to set the initial state for replicas of all existing partitions - * in zookeeper - */ - private def initializeReplicaState(): Unit = { - controllerContext.allPartitions.foreach { partition => - val replicas = controllerContext.partitionReplicaAssignment(partition) - replicas.foreach { replicaId => - val partitionAndReplica = PartitionAndReplica(partition, replicaId) - if (controllerContext.isReplicaOnline(replicaId, partition)) { - controllerContext.putReplicaState(partitionAndReplica, OnlineReplica) - } else { - // mark replicas on dead brokers as failed for topic deletion, if they belong to a topic to be deleted. - // This is required during controller failover since during controller failover a broker can go down, - // so the replicas on that broker should be moved to ReplicaDeletionIneligible to be on the safer side. - controllerContext.putReplicaState(partitionAndReplica, ReplicaDeletionIneligible) - } - } - } + throw new UnsupportedOperationException() } def handleStateChanges(replicas: Seq[PartitionAndReplica], targetState: ReplicaState): Unit @@ -106,350 +70,7 @@ class ZkReplicaStateMachine(config: KafkaConfig, this.logIdent = s"[ReplicaStateMachine controllerId=$controllerId] " override def handleStateChanges(replicas: Seq[PartitionAndReplica], targetState: ReplicaState): Unit = { - if (replicas.nonEmpty) { - try { - controllerBrokerRequestBatch.newBatch() - replicas.groupBy(_.replica).foreachEntry { (replicaId, replicas) => - doHandleStateChanges(replicaId, replicas, targetState) - } - controllerBrokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch) - } catch { - case e: ControllerMovedException => - error(s"Controller moved to another broker when moving some replicas to $targetState state", e) - throw e - case e: Throwable => error(s"Error while moving some replicas to $targetState state", e) - } - } - } - - /** - * This API exercises the replica's state machine. It ensures that every state transition happens from a legal - * previous state to the target state. Valid state transitions are: - * NonExistentReplica --> NewReplica - * --send LeaderAndIsr request with current leader and isr to the new replica and UpdateMetadata request for the - * partition to every live broker - * - * NewReplica -> OnlineReplica - * --add the new replica to the assigned replica list if needed - * - * OnlineReplica,OfflineReplica -> OnlineReplica - * --send LeaderAndIsr request with current leader and isr to the new replica and UpdateMetadata request for the - * partition to every live broker - * - * NewReplica,OnlineReplica,OfflineReplica,ReplicaDeletionIneligible -> OfflineReplica - * --send StopReplicaRequest to the replica (w/o deletion) - * --remove this replica from the isr and send LeaderAndIsr request (with new isr) to the leader replica and - * UpdateMetadata request for the partition to every live broker. - * - * OfflineReplica -> ReplicaDeletionStarted - * --send StopReplicaRequest to the replica (with deletion) - * - * ReplicaDeletionStarted -> ReplicaDeletionSuccessful - * -- mark the state of the replica in the state machine - * - * ReplicaDeletionStarted -> ReplicaDeletionIneligible - * -- mark the state of the replica in the state machine - * - * ReplicaDeletionSuccessful -> NonExistentReplica - * -- remove the replica from the in memory partition replica assignment cache - * - * @param replicaId The replica for which the state transition is invoked - * @param replicas The partitions on this replica for which the state transition is invoked - * @param targetState The end state that the replica should be moved to - */ - private def doHandleStateChanges(replicaId: Int, replicas: Seq[PartitionAndReplica], targetState: ReplicaState): Unit = { - val stateLogger = stateChangeLogger.withControllerEpoch(controllerContext.epoch) - val traceEnabled = stateLogger.isTraceEnabled - replicas.foreach(replica => controllerContext.putReplicaStateIfNotExists(replica, NonExistentReplica)) - val (validReplicas, invalidReplicas) = controllerContext.checkValidReplicaStateChange(replicas, targetState) - invalidReplicas.foreach(replica => logInvalidTransition(replica, targetState)) - - targetState match { - case NewReplica => - validReplicas.foreach { replica => - val partition = replica.topicPartition - val currentState = controllerContext.replicaState(replica) - - controllerContext.partitionLeadershipInfo(partition) match { - case Some(leaderIsrAndControllerEpoch) => - if (leaderIsrAndControllerEpoch.leaderAndIsr.leader == replicaId) { - val exception = new StateChangeFailedException(s"Replica $replicaId for partition $partition cannot be moved to NewReplica state as it is being requested to become leader") - logFailedStateChange(replica, currentState, OfflineReplica, exception) - } else { - controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(replicaId), - replica.topicPartition, - leaderIsrAndControllerEpoch, - controllerContext.partitionFullReplicaAssignment(replica.topicPartition), - isNew = true) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, partition, currentState, NewReplica) - controllerContext.putReplicaState(replica, NewReplica) - } - case None => - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, partition, currentState, NewReplica) - controllerContext.putReplicaState(replica, NewReplica) - } - } - case OnlineReplica => - validReplicas.foreach { replica => - val partition = replica.topicPartition - val currentState = controllerContext.replicaState(replica) - - currentState match { - case NewReplica => - val assignment = controllerContext.partitionFullReplicaAssignment(partition) - if (!assignment.replicas.contains(replicaId)) { - error(s"Adding replica ($replicaId) that is not part of the assignment $assignment") - val newAssignment = assignment.copy(replicas = assignment.replicas :+ replicaId) - controllerContext.updatePartitionFullReplicaAssignment(partition, newAssignment) - } - case _ => - controllerContext.partitionLeadershipInfo(partition) match { - case Some(leaderIsrAndControllerEpoch) => - controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(replicaId), - replica.topicPartition, - leaderIsrAndControllerEpoch, - controllerContext.partitionFullReplicaAssignment(partition), isNew = false) - case None => - } - } - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, partition, currentState, OnlineReplica) - controllerContext.putReplicaState(replica, OnlineReplica) - } - case OfflineReplica => - validReplicas.foreach { replica => - controllerBrokerRequestBatch.addStopReplicaRequestForBrokers(Seq(replicaId), replica.topicPartition, deletePartition = false) - } - val (replicasWithLeadershipInfo, replicasWithoutLeadershipInfo) = validReplicas.partition { replica => - controllerContext.partitionLeadershipInfo(replica.topicPartition).isDefined - } - val updatedLeaderIsrAndControllerEpochs = removeReplicasFromIsr(replicaId, replicasWithLeadershipInfo.map(_.topicPartition)) - updatedLeaderIsrAndControllerEpochs.foreachEntry { (partition, leaderIsrAndControllerEpoch) => - stateLogger.info(s"Partition $partition state changed to $leaderIsrAndControllerEpoch after removing replica $replicaId from the ISR as part of transition to $OfflineReplica") - if (!controllerContext.isTopicQueuedUpForDeletion(partition.topic)) { - val recipients = controllerContext.partitionReplicaAssignment(partition).filterNot(_ == replicaId) - controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(recipients, - partition, - leaderIsrAndControllerEpoch, - controllerContext.partitionFullReplicaAssignment(partition), isNew = false) - } - val replica = PartitionAndReplica(partition, replicaId) - val currentState = controllerContext.replicaState(replica) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, partition, currentState, OfflineReplica) - controllerContext.putReplicaState(replica, OfflineReplica) - } - - replicasWithoutLeadershipInfo.foreach { replica => - val currentState = controllerContext.replicaState(replica) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, replica.topicPartition, currentState, OfflineReplica) - controllerBrokerRequestBatch.addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(replica.topicPartition)) - controllerContext.putReplicaState(replica, OfflineReplica) - } - case ReplicaDeletionStarted => - validReplicas.foreach { replica => - val currentState = controllerContext.replicaState(replica) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, replica.topicPartition, currentState, ReplicaDeletionStarted) - controllerContext.putReplicaState(replica, ReplicaDeletionStarted) - controllerBrokerRequestBatch.addStopReplicaRequestForBrokers(Seq(replicaId), replica.topicPartition, deletePartition = true) - } - case ReplicaDeletionIneligible => - validReplicas.foreach { replica => - val currentState = controllerContext.replicaState(replica) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, replica.topicPartition, currentState, ReplicaDeletionIneligible) - controllerContext.putReplicaState(replica, ReplicaDeletionIneligible) - } - case ReplicaDeletionSuccessful => - validReplicas.foreach { replica => - val currentState = controllerContext.replicaState(replica) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, replica.topicPartition, currentState, ReplicaDeletionSuccessful) - controllerContext.putReplicaState(replica, ReplicaDeletionSuccessful) - } - case NonExistentReplica => - validReplicas.foreach { replica => - val currentState = controllerContext.replicaState(replica) - val newAssignedReplicas = controllerContext - .partitionFullReplicaAssignment(replica.topicPartition) - .removeReplica(replica.replica) - - controllerContext.updatePartitionFullReplicaAssignment(replica.topicPartition, newAssignedReplicas) - if (traceEnabled) - logSuccessfulTransition(stateLogger, replicaId, replica.topicPartition, currentState, NonExistentReplica) - controllerContext.removeReplicaState(replica) - } - } - } - - /** - * Repeatedly attempt to remove a replica from the isr of multiple partitions until there are no more remaining partitions - * to retry. - * @param replicaId The replica being removed from isr of multiple partitions - * @param partitions The partitions from which we're trying to remove the replica from isr - * @return The updated LeaderIsrAndControllerEpochs of all partitions for which we successfully removed the replica from isr. - */ - private def removeReplicasFromIsr( - replicaId: Int, - partitions: Seq[TopicPartition] - ): Map[TopicPartition, LeaderIsrAndControllerEpoch] = { - var results = Map.empty[TopicPartition, LeaderIsrAndControllerEpoch] - var remaining = partitions - while (remaining.nonEmpty) { - val (finishedRemoval, removalsToRetry) = doRemoveReplicasFromIsr(replicaId, remaining) - remaining = removalsToRetry - - finishedRemoval.foreach { - case (partition, Left(e)) => - val replica = PartitionAndReplica(partition, replicaId) - val currentState = controllerContext.replicaState(replica) - logFailedStateChange(replica, currentState, OfflineReplica, e) - case (partition, Right(leaderIsrAndEpoch)) => - results += partition -> leaderIsrAndEpoch - } - } - results - } - - /** - * Try to remove a replica from the isr of multiple partitions. - * Removing a replica from isr updates partition state in zookeeper. - * - * @param replicaId The replica being removed from isr of multiple partitions - * @param partitions The partitions from which we're trying to remove the replica from isr - * @return A tuple of two elements: - * 1. The updated Right[LeaderIsrAndControllerEpochs] of all partitions for which we successfully - * removed the replica from isr. Or Left[Exception] corresponding to failed removals that should - * not be retried - * 2. The partitions that we should retry due to a zookeeper BADVERSION conflict. Version conflicts can occur if - * the partition leader updated partition state while the controller attempted to update partition state. - */ - private def doRemoveReplicasFromIsr( - replicaId: Int, - partitions: Seq[TopicPartition] - ): (Map[TopicPartition, Either[Exception, LeaderIsrAndControllerEpoch]], Seq[TopicPartition]) = { - val (leaderAndIsrs, partitionsWithNoLeaderAndIsrInZk) = getTopicPartitionStatesFromZk(partitions) - val (leaderAndIsrsWithReplica, leaderAndIsrsWithoutReplica) = leaderAndIsrs.partition { case (_, result) => - result.map { leaderAndIsr => - leaderAndIsr.isr.contains(replicaId) - }.getOrElse(false) - } - - val adjustedLeaderAndIsrs: Map[TopicPartition, LeaderAndIsr] = leaderAndIsrsWithReplica.flatMap { - case (partition, result) => - result.toOption.map { leaderAndIsr => - val newLeader = if (replicaId == leaderAndIsr.leader) LeaderAndIsr.NO_LEADER else leaderAndIsr.leader - val adjustedIsr = - if (leaderAndIsr.isr.size == 1) leaderAndIsr.isr - else leaderAndIsr.isr.stream().filter(_ != replicaId).collect(Collectors.toList[Integer]) - partition -> leaderAndIsr.newLeaderAndIsr(newLeader, adjustedIsr) - } - } - - val UpdateLeaderAndIsrResult(finishedPartitions, updatesToRetry) = zkClient.updateLeaderAndIsr( - adjustedLeaderAndIsrs, controllerContext.epoch, controllerContext.epochZkVersion) - - val exceptionsForPartitionsWithNoLeaderAndIsrInZk: Map[TopicPartition, Either[Exception, LeaderIsrAndControllerEpoch]] = - partitionsWithNoLeaderAndIsrInZk.iterator.flatMap { partition => - if (!controllerContext.isTopicQueuedUpForDeletion(partition.topic)) { - val exception = new StateChangeFailedException( - s"Failed to change state of replica $replicaId for partition $partition since the leader and isr " + - "path in zookeeper is empty" - ) - Option(partition -> Left(exception)) - } else None - }.toMap - - val leaderIsrAndControllerEpochs: Map[TopicPartition, Either[Exception, LeaderIsrAndControllerEpoch]] = - (leaderAndIsrsWithoutReplica ++ finishedPartitions).map { case (partition, result) => - (partition, result.map { leaderAndIsr => - val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) - controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) - leaderIsrAndControllerEpoch - }) - } - - if (isDebugEnabled) { - updatesToRetry.foreach { partition => - debug(s"Controller failed to remove replica $replicaId from ISR of partition $partition. " + - s"Attempted to write state ${adjustedLeaderAndIsrs(partition)}, but failed with bad ZK version. This will be retried.") - } - } - - (leaderIsrAndControllerEpochs ++ exceptionsForPartitionsWithNoLeaderAndIsrInZk, updatesToRetry) - } - - /** - * Gets the partition state from zookeeper - * @param partitions the partitions whose state we want from zookeeper - * @return A tuple of two values: - * 1. The Right(LeaderAndIsrs) of partitions whose state we successfully read from zookeeper. - * The Left(Exception) to failed zookeeper lookups or states whose controller epoch exceeds our current epoch - * 2. The partitions that had no leader and isr state in zookeeper. This happens if the controller - * didn't finish partition initialization. - */ - private def getTopicPartitionStatesFromZk( - partitions: Seq[TopicPartition] - ): (Map[TopicPartition, Either[Exception, LeaderAndIsr]], Seq[TopicPartition]) = { - val getDataResponses = try { - zkClient.getTopicPartitionStatesRaw(partitions) - } catch { - case e: Exception => - return (partitions.iterator.map(_ -> Left(e)).toMap, Seq.empty) - } - - val partitionsWithNoLeaderAndIsrInZk = mutable.Buffer.empty[TopicPartition] - val result = mutable.Map.empty[TopicPartition, Either[Exception, LeaderAndIsr]] - - getDataResponses.foreach[Unit] { getDataResponse => - val partition = getDataResponse.ctx.get.asInstanceOf[TopicPartition] - if (getDataResponse.resultCode == Code.OK) { - TopicPartitionStateZNode.decode(getDataResponse.data, getDataResponse.stat) match { - case None => - partitionsWithNoLeaderAndIsrInZk += partition - case Some(leaderIsrAndControllerEpoch) => - if (leaderIsrAndControllerEpoch.controllerEpoch > controllerContext.epoch) { - val exception = new StateChangeFailedException( - "Leader and isr path written by another controller. This probably " + - s"means the current controller with epoch ${controllerContext.epoch} went through a soft failure and " + - s"another controller was elected with epoch ${leaderIsrAndControllerEpoch.controllerEpoch}. Aborting " + - "state change by this controller" - ) - result += (partition -> Left(exception)) - } else { - result += (partition -> Right(leaderIsrAndControllerEpoch.leaderAndIsr)) - } - } - } else if (getDataResponse.resultCode == Code.NONODE) { - partitionsWithNoLeaderAndIsrInZk += partition - } else { - result += (partition -> Left(getDataResponse.resultException.get)) - } - } - - (result.toMap, partitionsWithNoLeaderAndIsrInZk) - } - - private def logSuccessfulTransition(logger: StateChangeLogger, replicaId: Int, partition: TopicPartition, - currState: ReplicaState, targetState: ReplicaState): Unit = { - logger.trace(s"Changed state of replica $replicaId for partition $partition from $currState to $targetState") - } - - private def logInvalidTransition(replica: PartitionAndReplica, targetState: ReplicaState): Unit = { - val currState = controllerContext.replicaState(replica) - val e = new IllegalStateException(s"Replica $replica should be in the ${targetState.validPreviousStates.mkString(",")} " + - s"states before moving to $targetState state. Instead it is in $currState state") - logFailedStateChange(replica, currState, targetState, e) - } - - private def logFailedStateChange(replica: PartitionAndReplica, currState: ReplicaState, targetState: ReplicaState, t: Throwable): Unit = { - stateChangeLogger.withControllerEpoch(controllerContext.epoch) - .error(s"Controller $controllerId epoch ${controllerContext.epoch} initiated state change of replica ${replica.replica} " + - s"for partition ${replica.topicPartition} from $currState to $targetState failed", t) + throw new UnsupportedOperationException() } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 1db7d1231fd..11b0fa6bfda 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -49,7 +49,6 @@ import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.server.util.Csv import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} -import org.apache.zookeeper.client.ZKClientConfig import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} @@ -62,18 +61,6 @@ object KafkaConfig { DynamicBrokerConfig.dynamicConfigUpdateModes)) } - private[kafka] def zooKeeperClientProperty(clientConfig: ZKClientConfig, kafkaPropName: String): Option[String] = { - Option(clientConfig.getProperty(ZkConfigs.ZK_SSL_CONFIG_TO_SYSTEM_PROPERTY_MAP.get(kafkaPropName))) - } - - // For ZooKeeper TLS client authentication to be enabled the client must (at a minimum) configure itself as using TLS - // with both a client connection socket and a key store location explicitly set. - private[kafka] def zkTlsClientAuthEnabled(zkClientConfig: ZKClientConfig): Boolean = { - zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_CLIENT_ENABLE_CONFIG).contains("true") && - zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_CLIENT_CNXN_SOCKET_CONFIG).isDefined && - zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG).isDefined - } - val configDef = AbstractKafkaConfig.CONFIG_DEF def configNames: Seq[String] = configDef.names.asScala.toBuffer.sorted diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index 290802c5d11..5c6ee6e8512 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -18,7 +18,6 @@ package kafka.zk import java.util.{Collections, Optional, Properties} import kafka.admin.RackAwareMode -import kafka.common.TopicAlreadyMarkedForDeletionException import kafka.controller.ReplicaAssignment import kafka.server.{DynamicConfig, KafkaConfig} import kafka.utils._ @@ -29,7 +28,6 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.server.common.AdminOperationException import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals} import org.apache.kafka.storage.internals.log.LogConfig -import org.apache.zookeeper.KeeperException.NodeExistsException import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} @@ -180,7 +178,6 @@ class AdminZkClient(zkClient: KafkaZkClient, } debug("Updated path %s with %s for replica assignment".format(TopicZNode.path(topic), assignment)) } catch { - case _: NodeExistsException => throw new TopicExistsException(s"Topic '$topic' already exists.") case e2: Throwable => throw new AdminOperationException(e2.toString) } } @@ -194,8 +191,6 @@ class AdminZkClient(zkClient: KafkaZkClient, try { zkClient.createDeleteTopicPath(topic) } catch { - case _: NodeExistsException => throw new TopicAlreadyMarkedForDeletionException( - "topic %s is already marked for deletion".format(topic)) case e: Throwable => throw new AdminOperationException(e.getMessage) } } else { diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 8efa12ee7fd..e03818f0be3 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -18,32 +18,16 @@ package kafka.zk import java.util.Properties import kafka.cluster.Broker -import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReplicaAssignment} -import kafka.server.KafkaConfig +import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.utils.Logging import kafka.zk.TopicZNode.TopicIdReplicaAssignment import kafka.zookeeper._ -import org.apache.kafka.common.errors.ControllerMovedException -import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} -import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} -import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} +import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderAndIsr -import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.{ConfigType, ZkConfigs} -import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.storage.internals.log.LogConfig -import org.apache.zookeeper.KeeperException.{Code, NodeExistsException} -import org.apache.zookeeper.OpResult.{CreateResult, ErrorResult, SetDataResult} -import org.apache.zookeeper.client.ZKClientConfig -import org.apache.zookeeper.common.ZKConfig -import org.apache.zookeeper.data.{ACL, Stat} -import org.apache.zookeeper.{CreateMode, KeeperException, OpResult, ZooKeeper} -import java.lang.{Long => JLong} -import scala.collection.{Map, Seq, mutable} -import scala.jdk.CollectionConverters._ +import scala.collection.{Map, Seq} /** * Provides higher level Kafka-specific operations on top of the pipelined [[kafka.zookeeper.ZooKeeperClient]]. @@ -53,28 +37,9 @@ import scala.jdk.CollectionConverters._ * easier to migrate away from `ZkUtils` (since removed). We should revisit this. We should also consider whether a * monolithic [[kafka.zk.ZkData]] is the way to go. */ -class KafkaZkClient private[zk] ( - zooKeeperClient: ZooKeeperClient, - isSecure: Boolean, - time: Time, - enableEntityConfigControllerCheck: Boolean -) extends AutoCloseable with Logging { - - private val metricsGroup: KafkaMetricsGroup = new KafkaMetricsGroup("kafka.server", "ZooKeeperClientMetrics") - - private val latencyMetric = metricsGroup.newHistogram("ZooKeeperRequestLatencyMs") - +class KafkaZkClient() extends AutoCloseable with Logging { import KafkaZkClient._ - // Only for testing - private[kafka] def currentZooKeeper: ZooKeeper = zooKeeperClient.currentZooKeeper - - // This variable holds the Zookeeper session id at the moment a Broker gets registered in Zookeeper and the subsequent - // updates of the session id. It is possible that the session id changes over the time for 'Session expired'. - // This code is part of the work around done in the KAFKA-7165, once ZOOKEEPER-2985 is complete, this code must - // be deleted. - private var currentZooKeeperSessionId: Long = -1 - /** * Create a sequential persistent path. That is, the znode will not be automatically deleted upon client's disconnect * and a monotonically increasing number will be appended to its name. @@ -84,10 +49,7 @@ class KafkaZkClient private[zk] ( * @return the created path (including the appended monotonically increasing number) */ private[kafka] def createSequentialPersistentPath(path: String, data: Array[Byte]): String = { - val createRequest = CreateRequest(path, data, defaultAcls(path), CreateMode.PERSISTENT_SEQUENTIAL) - val createResponse = retryRequestUntilConnected(createRequest) - createResponse.maybeThrow() - createResponse.name + throw new UnsupportedOperationException() } /** @@ -96,89 +58,20 @@ class KafkaZkClient private[zk] ( * @return broker epoch (znode create transaction id) */ def registerBroker(brokerInfo: BrokerInfo): Long = { - val path = brokerInfo.path - val stat = checkedEphemeralCreate(path, brokerInfo.toJsonBytes) - info(s"Registered broker ${brokerInfo.broker.id} at path $path with addresses: " + - s"${brokerInfo.broker.endPoints.map(_.connectionString).mkString(",")}, czxid (broker epoch): ${stat.getCzxid}") - stat.getCzxid + throw new UnsupportedOperationException() } /** * Registers a given broker in zookeeper as the controller and increments controller epoch. * @param controllerId the id of the broker that is to be registered as the controller. * @return the (updated controller epoch, epoch zkVersion) tuple - * @throws ControllerMovedException if fail to create /controller or fail to increment controller epoch. */ def registerControllerAndIncrementControllerEpoch(controllerId: Int): (Int, Int) = { - val timestamp = time.milliseconds() - - // Read /controller_epoch to get the current controller epoch and zkVersion, - // create /controller_epoch with initial value if not exists - val (curEpoch, curEpochZkVersion) = getControllerEpoch - .map(e => (e._1, e._2.getVersion)) - .getOrElse(maybeCreateControllerEpochZNode()) - - // Create /controller and update /controller_epoch atomically - val newControllerEpoch = curEpoch + 1 - val expectedControllerEpochZkVersion = curEpochZkVersion - - debug(s"Try to create ${ControllerZNode.path} and increment controller epoch to $newControllerEpoch with expected controller epoch zkVersion $expectedControllerEpochZkVersion") - - def checkControllerAndEpoch(): (Int, Int) = { - val curControllerId = getControllerId.getOrElse(throw new ControllerMovedException( - s"The ephemeral node at ${ControllerZNode.path} went away while checking whether the controller election succeeds. " + - s"Aborting controller startup procedure")) - if (controllerId == curControllerId) { - val (epoch, stat) = getControllerEpoch.getOrElse( - throw new IllegalStateException(s"${ControllerEpochZNode.path} existed before but goes away while trying to read it")) - - // If the epoch is the same as newControllerEpoch, it is safe to infer that the returned epoch zkVersion - // is associated with the current broker during controller election because we already knew that the zk - // transaction succeeds based on the controller znode verification. Other rounds of controller - // election will result in larger epoch number written in zk. - if (epoch == newControllerEpoch) - return (newControllerEpoch, stat.getVersion) - } - throw new ControllerMovedException("Controller moved to another broker. Aborting controller startup procedure") - } - - def tryCreateControllerZNodeAndIncrementEpoch(): (Int, Int) = { - val response = retryRequestUntilConnected( - MultiRequest(Seq( - CreateOp(ControllerZNode.path, ControllerZNode.encode(controllerId, timestamp), defaultAcls(ControllerZNode.path), CreateMode.EPHEMERAL), - SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), expectedControllerEpochZkVersion))) - ) - response.resultCode match { - case Code.NODEEXISTS | Code.BADVERSION => checkControllerAndEpoch() - case Code.OK => - val setDataResult = response.zkOpResults(1).rawOpResult.asInstanceOf[SetDataResult] - (newControllerEpoch, setDataResult.getStat.getVersion) - case code => throw KeeperException.create(code) - } - } - - tryCreateControllerZNodeAndIncrementEpoch() - } - - private def maybeCreateControllerEpochZNode(): (Int, Int) = { - createControllerEpochRaw(KafkaController.InitialControllerEpoch).resultCode match { - case Code.OK => - info(s"Successfully created ${ControllerEpochZNode.path} with initial epoch ${KafkaController.InitialControllerEpoch}") - (KafkaController.InitialControllerEpoch, KafkaController.InitialControllerEpochZkVersion) - case Code.NODEEXISTS => - val (epoch, stat) = getControllerEpoch.getOrElse(throw new IllegalStateException(s"${ControllerEpochZNode.path} existed before but goes away while trying to read it")) - (epoch, stat.getVersion) - case code => - throw KeeperException.create(code) - } + throw new UnsupportedOperationException() } def updateBrokerInfo(brokerInfo: BrokerInfo): Unit = { - val brokerIdPath = brokerInfo.path - val setDataRequest = SetDataRequest(brokerIdPath, brokerInfo.toJsonBytes, ZkVersion.MatchAnyVersion) - val response = retryRequestUntilConnected(setDataRequest) - response.maybeThrow() - info("Updated broker %d at path %s with addresses: %s".format(brokerInfo.broker.id, brokerIdPath, brokerInfo.broker.endPoints)) + throw new UnsupportedOperationException() } /** @@ -187,10 +80,7 @@ class KafkaZkClient private[zk] ( * @return sequence of GetDataResponses whose contexts are the partitions they are associated with. */ def getTopicPartitionStatesRaw(partitions: Seq[TopicPartition]): Seq[GetDataResponse] = { - val getDataRequests = partitions.map { partition => - GetDataRequest(TopicPartitionStateZNode.path(partition), ctx = Some(partition)) - } - retryRequestsUntilConnected(getDataRequests) + throw new UnsupportedOperationException() } /** @@ -200,12 +90,7 @@ class KafkaZkClient private[zk] ( * @return sequence of SetDataResponse whose contexts are the partitions they are associated with. */ def setTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[SetDataResponse] = { - val setDataRequests = leaderIsrAndControllerEpochs.map { case (partition, leaderIsrAndControllerEpoch) => - val path = TopicPartitionStateZNode.path(partition) - val data = TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch) - SetDataRequest(path, data, leaderIsrAndControllerEpoch.leaderAndIsr.partitionEpoch, Some(partition)) - } - retryRequestsUntilConnected(setDataRequests.toSeq, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -215,14 +100,7 @@ class KafkaZkClient private[zk] ( * @return sequence of CreateResponse whose contexts are the partitions they are associated with. */ def createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[CreateResponse] = { - createTopicPartitions(leaderIsrAndControllerEpochs.keys.map(_.topic).toSeq.distinct, expectedControllerEpochZkVersion) - createTopicPartition(leaderIsrAndControllerEpochs.keys.toSeq, expectedControllerEpochZkVersion) - val createRequests = leaderIsrAndControllerEpochs.map { case (partition, leaderIsrAndControllerEpoch) => - val path = TopicPartitionStateZNode.path(partition) - val data = TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch) - CreateRequest(path, data, defaultAcls(path), CreateMode.PERSISTENT, Some(partition)) - } - retryRequestsUntilConnected(createRequests.toSeq, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -232,8 +110,7 @@ class KafkaZkClient private[zk] ( * @return SetDataResponse */ def setControllerEpochRaw(epoch: Int, epochZkVersion: Int): SetDataResponse = { - val setDataRequest = SetDataRequest(ControllerEpochZNode.path, ControllerEpochZNode.encode(epoch), epochZkVersion) - retryRequestUntilConnected(setDataRequest) + throw new UnsupportedOperationException() } /** @@ -242,9 +119,7 @@ class KafkaZkClient private[zk] ( * @return CreateResponse */ def createControllerEpochRaw(epoch: Int): CreateResponse = { - val createRequest = CreateRequest(ControllerEpochZNode.path, ControllerEpochZNode.encode(epoch), - defaultAcls(ControllerEpochZNode.path), CreateMode.PERSISTENT) - retryRequestUntilConnected(createRequest) + throw new UnsupportedOperationException() } /** @@ -259,34 +134,7 @@ class KafkaZkClient private[zk] ( controllerEpoch: Int, expectedControllerEpochZkVersion: Int ): UpdateLeaderAndIsrResult = { - val leaderIsrAndControllerEpochs = leaderAndIsrs.map { case (partition, leaderAndIsr) => - partition -> LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) - } - val setDataResponses = try { - setTopicPartitionStatesRaw(leaderIsrAndControllerEpochs, expectedControllerEpochZkVersion) - } catch { - case e: ControllerMovedException => throw e - case e: Exception => - return UpdateLeaderAndIsrResult(leaderAndIsrs.keys.iterator.map(_ -> Left(e)).toMap, Seq.empty) - } - - val updatesToRetry = mutable.Buffer.empty[TopicPartition] - val finished = setDataResponses.iterator.flatMap { setDataResponse => - val partition = setDataResponse.ctx.get.asInstanceOf[TopicPartition] - setDataResponse.resultCode match { - case Code.OK => - val updatedLeaderAndIsr = leaderAndIsrs(partition).withPartitionEpoch(setDataResponse.stat.getVersion) - Some(partition -> Right(updatedLeaderAndIsr)) - case Code.BADVERSION => - // Update the buffer for partitions to retry - updatesToRetry += partition - None - case _ => - Some(partition -> Left(setDataResponse.resultException.get)) - } - }.toMap - - UpdateLeaderAndIsrResult(finished, updatesToRetry) + throw new UnsupportedOperationException() } /** @@ -301,29 +149,7 @@ class KafkaZkClient private[zk] ( topics: Set[String], config: java.util.Map[String, AnyRef] ): (Map[String, LogConfig], Map[String, Exception]) = { - val logConfigs = mutable.Map.empty[String, LogConfig] - val failed = mutable.Map.empty[String, Exception] - val configResponses = try { - getTopicConfigs(topics) - } catch { - case e: Exception => - topics.foreach(topic => failed.put(topic, e)) - return (logConfigs.toMap, failed.toMap) - } - configResponses.foreach { configResponse => - val topic = configResponse.ctx.get.asInstanceOf[String] - configResponse.resultCode match { - case Code.OK => - val overrides = ConfigEntityZNode.decode(configResponse.data) - val logConfig = LogConfig.fromProps(config, overrides) - logConfigs.put(topic, logConfig) - case Code.NONODE => - val logConfig = LogConfig.fromProps(config, new Properties) - logConfigs.put(topic, logConfig) - case _ => failed.put(topic, configResponse.resultException.get) - } - } - (logConfigs.toMap, failed.toMap) + throw new UnsupportedOperationException() } /** @@ -333,33 +159,11 @@ class KafkaZkClient private[zk] ( * @return The successfully gathered log configs */ def getEntityConfigs(rootEntityType: String, sanitizedEntityName: String): Properties = { - val getDataRequest = GetDataRequest(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName)) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - - getDataResponse.resultCode match { - case Code.OK => - ConfigEntityZNode.decode(getDataResponse.data) - case Code.NONODE => new Properties() - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } def getEntitiesConfigs(rootEntityType: String, sanitizedEntityNames: Set[String]): Map[String, Properties] = { - val getDataRequests: Seq[GetDataRequest] = sanitizedEntityNames.map { entityName => - GetDataRequest(ConfigEntityZNode.path(rootEntityType, entityName), Some(entityName)) - }.toSeq - - val getDataResponses = retryRequestsUntilConnected(getDataRequests) - getDataResponses.map { response => - val entityName = response.ctx.get.asInstanceOf[String] - response.resultCode match { - case Code.OK => - entityName -> ConfigEntityZNode.decode(response.data) - case Code.NONODE => - entityName -> new Properties() - case _ => throw response.resultException.get - } - }.toMap + throw new UnsupportedOperationException() } /** @@ -382,67 +186,9 @@ class KafkaZkClient private[zk] ( * @param rootEntityType entity type * @param sanitizedEntityName entity name * @throws KeeperException if there is an error while setting or creating the znode - * @throws ControllerMovedException if no controller is defined, or a KRaft controller is defined */ def setOrCreateEntityConfigs(rootEntityType: String, sanitizedEntityName: String, config: Properties): Unit = { - val controllerZkVersionOpt: Option[Int] = if (enableEntityConfigControllerCheck) { - val controllerRegistration = getControllerRegistration match { - case Some(registration) => registration - case None => - // This case is mainly here to make tests less flaky (by virtue of retries). - // In practice, we always expect a /controller ZNode to exist - throw new ControllerMovedException(s"Cannot set entity configs for $rootEntityType $sanitizedEntityName " + - s"when there is no controller.") - } - - // If there is a KRaft controller defined, don't even attempt this write. The broker will soon get a UMR - // from the new KRaft controller that lets it know about the new controller. It will then forward - // IncrementalAlterConfig requests instead of processing directly. - if (controllerRegistration.kraftEpoch.exists(epoch => epoch > 0)) { - throw new ControllerMovedException(s"Cannot set entity configs for $rootEntityType $sanitizedEntityName " + - s"directly when there is a KRaft controller.") - } - - Some(controllerRegistration.zkVersion) - } else { - logger.warn("Setting entity configs without any checks on the controller.") - None - } - - def set(configData: Array[Byte]): SetDataResponse = { - // Since we're guarding against the controller switching to KRaft, we need to check that the controller hasn't - // changed during this method. We do that here by adding a CheckOp on the controller ZNode. The reason we - // don't use the controller epoch zkVersion here is that we can't consistently read the controller and - // controller epoch. This does _not_ guard against the existing "last writer wins" behavior of this method. - controllerZkVersionOpt match { - case Some(controllerZkVersion) => - val multi = MultiRequest(Seq( - CheckOp(ControllerZNode.path, controllerZkVersion), - SetDataOp(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName), configData, ZkVersion.MatchAnyVersion) - )) - val results = retryRequestUntilConnected(multi) - unwrapResponseWithControllerEpochCheck(results).asInstanceOf[SetDataResponse] - case None => - val setDataRequest = SetDataRequest(ConfigEntityZNode.path(rootEntityType, sanitizedEntityName), configData, ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(setDataRequest) - } - } - - def createOrSet(configData: Array[Byte]): Unit = { - val path = ConfigEntityZNode.path(rootEntityType, sanitizedEntityName) - try createRecursive(path, configData) - catch { - case _: NodeExistsException => set(configData).maybeThrow() - } - } - - val configData = ConfigEntityZNode.encode(config) - - val setDataResponse = set(configData) - setDataResponse.resultCode match { - case Code.NONODE => createOrSet(configData) - case _ => setDataResponse.maybeThrow() - } + throw new UnsupportedOperationException() } /** @@ -451,7 +197,7 @@ class KafkaZkClient private[zk] ( * @return List of all entity names */ def getAllEntitiesWithConfig(entityType: String): Seq[String] = { - getChildren(ConfigEntityTypeZNode.path(entityType)) + throw new UnsupportedOperationException() } /** @@ -460,11 +206,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while setting or creating the znode */ def createConfigChangeNotification(sanitizedEntityPath: String): Unit = { - makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path) - val path = ConfigEntityChangeNotificationSequenceZNode.createPath - val createRequest = CreateRequest(path, ConfigEntityChangeNotificationSequenceZNode.encode(sanitizedEntityPath), defaultAcls(path), CreateMode.PERSISTENT_SEQUENTIAL) - val createResponse = retryRequestUntilConnected(createRequest) - createResponse.maybeThrow() + throw new UnsupportedOperationException() } /** @@ -472,18 +214,7 @@ class KafkaZkClient private[zk] ( * @return sequence of brokers in the cluster. */ def getAllBrokersInCluster: Seq[Broker] = { - val brokerIds = getSortedBrokerList - val getDataRequests = brokerIds.map(brokerId => GetDataRequest(BrokerIdZNode.path(brokerId), ctx = Some(brokerId))) - val getDataResponses = retryRequestsUntilConnected(getDataRequests) - getDataResponses.flatMap { getDataResponse => - val brokerId = getDataResponse.ctx.get.asInstanceOf[Int] - getDataResponse.resultCode match { - case Code.OK => - Option(BrokerIdZNode.decode(brokerId, getDataResponse.data).broker) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - } + throw new UnsupportedOperationException() } /** @@ -491,18 +222,7 @@ class KafkaZkClient private[zk] ( * @return map of broker to epoch in the cluster. */ def getAllBrokerAndEpochsInCluster: Map[Broker, Long] = { - val brokerIds = getSortedBrokerList - val getDataRequests = brokerIds.map(brokerId => GetDataRequest(BrokerIdZNode.path(brokerId), ctx = Some(brokerId))) - val getDataResponses = retryRequestsUntilConnected(getDataRequests) - getDataResponses.flatMap { getDataResponse => - val brokerId = getDataResponse.ctx.get.asInstanceOf[Int] - getDataResponse.resultCode match { - case Code.OK => - Some((BrokerIdZNode.decode(brokerId, getDataResponse.data).broker, getDataResponse.stat.getCzxid)) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - }.toMap + throw new UnsupportedOperationException() } /** @@ -510,20 +230,15 @@ class KafkaZkClient private[zk] ( * @return an optional Broker */ def getBroker(brokerId: Int): Option[Broker] = { - val getDataRequest = GetDataRequest(BrokerIdZNode.path(brokerId)) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => - Option(BrokerIdZNode.decode(brokerId, getDataResponse.data).broker) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** * Gets the list of sorted broker Ids */ - def getSortedBrokerList: Seq[Int] = getChildren(BrokerIdsZNode.path).map(_.toInt).sorted + def getSortedBrokerList: Seq[Int] = { + throw new UnsupportedOperationException() + } /** * Gets all topics in the cluster. @@ -531,13 +246,7 @@ class KafkaZkClient private[zk] ( * @return sequence of topics in the cluster. */ def getAllTopicsInCluster(registerWatch: Boolean = false): Set[String] = { - val getChildrenResponse = retryRequestUntilConnected( - GetChildrenRequest(TopicsZNode.path, registerWatch)) - getChildrenResponse.resultCode match { - case Code.OK => getChildrenResponse.children.toSet - case Code.NONODE => Set.empty - case _ => throw getChildrenResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -546,7 +255,7 @@ class KafkaZkClient private[zk] ( * @return true if topic exists else false */ def topicExists(topicName: String): Boolean = { - pathExists(TopicZNode.path(topicName)) + throw new UnsupportedOperationException() } /** @@ -556,19 +265,7 @@ class KafkaZkClient private[zk] ( */ def setTopicIds(topicIdReplicaAssignments: collection.Set[TopicIdReplicaAssignment], expectedControllerEpochZkVersion: Int): Set[TopicIdReplicaAssignment] = { - val updatedAssignments = topicIdReplicaAssignments.map { - case TopicIdReplicaAssignment(topic, None, assignments) => - TopicIdReplicaAssignment(topic, Some(Uuid.randomUuid()), assignments) - case TopicIdReplicaAssignment(topic, Some(_), _) => - throw new IllegalArgumentException("TopicIdReplicaAssignment for " + topic + " already contains a topic ID.") - }.toSet - - val setDataRequests = updatedAssignments.map { case TopicIdReplicaAssignment(topic, topicIdOpt, assignments) => - SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(topicIdOpt, assignments), ZkVersion.MatchAnyVersion) - }.toSeq - - retryRequestsUntilConnected(setDataRequests, expectedControllerEpochZkVersion) - updatedAssignments + throw new UnsupportedOperationException() } /** @@ -583,8 +280,7 @@ class KafkaZkClient private[zk] ( topicId: Option[Uuid], assignment: collection.Map[TopicPartition, ReplicaAssignment], expectedControllerEpochZkVersion: Int): SetDataResponse = { - val setDataRequest = SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(topicId, assignment), ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(setDataRequest, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -599,8 +295,7 @@ class KafkaZkClient private[zk] ( topicId: Option[Uuid], assignment: Map[TopicPartition, ReplicaAssignment], expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion): Unit = { - val setDataResponse = setTopicAssignmentRaw(topic, topicId, assignment, expectedControllerEpochZkVersion) - setDataResponse.maybeThrow() + throw new UnsupportedOperationException() } /** @@ -611,8 +306,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while creating assignment */ def createTopicAssignment(topic: String, topicId: Option[Uuid], assignment: Map[TopicPartition, Seq[Int]]): Unit = { - val persistedAssignments = assignment.map { case (k, v) => k -> ReplicaAssignment(v) } - createRecursive(TopicZNode.path(topic), TopicZNode.encode(topicId, persistedAssignments)) + throw new UnsupportedOperationException() } /** @@ -620,12 +314,7 @@ class KafkaZkClient private[zk] ( * @return sequence of znode names and not the absolute znode path. */ def getAllLogDirEventNotifications: Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(LogDirEventNotificationZNode.path, registerWatch = true)) - getChildrenResponse.resultCode match { - case Code.OK => getChildrenResponse.children.map(LogDirEventNotificationSequenceZNode.sequenceNumber) - case Code.NONODE => Seq.empty - case _ => throw getChildrenResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -634,17 +323,7 @@ class KafkaZkClient private[zk] ( * @return broker ids associated with the given log dir event notifications. */ def getBrokerIdsFromLogDirEvents(sequenceNumbers: Seq[String]): Seq[Int] = { - val getDataRequests = sequenceNumbers.map { sequenceNumber => - GetDataRequest(LogDirEventNotificationSequenceZNode.path(sequenceNumber)) - } - val getDataResponses = retryRequestsUntilConnected(getDataRequests) - getDataResponses.flatMap { getDataResponse => - getDataResponse.resultCode match { - case Code.OK => LogDirEventNotificationSequenceZNode.decode(getDataResponse.data) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - } + throw new UnsupportedOperationException() } /** @@ -652,12 +331,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteLogDirEventNotifications(expectedControllerEpochZkVersion: Int): Unit = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(LogDirEventNotificationZNode.path, registerWatch = true)) - if (getChildrenResponse.resultCode == Code.OK) { - deleteLogDirEventNotifications(getChildrenResponse.children.map(LogDirEventNotificationSequenceZNode.sequenceNumber), expectedControllerEpochZkVersion) - } else if (getChildrenResponse.resultCode != Code.NONODE) { - getChildrenResponse.maybeThrow() - } + throw new UnsupportedOperationException() } /** @@ -666,10 +340,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteLogDirEventNotifications(sequenceNumbers: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - val deleteRequests = sequenceNumbers.map { sequenceNumber => - DeleteRequest(LogDirEventNotificationSequenceZNode.path(sequenceNumber), ZkVersion.MatchAnyVersion) - } - retryRequestsUntilConnected(deleteRequests, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -678,19 +349,7 @@ class KafkaZkClient private[zk] ( * @return the Topic IDs */ def getTopicIdsForTopics(topics: Set[String]): Map[String, Uuid] = { - val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic))) - val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq) - getDataResponses.map { getDataResponse => - val topic = getDataResponse.ctx.get.asInstanceOf[String] - getDataResponse.resultCode match { - case Code.OK => Some(TopicZNode.decode(topic, getDataResponse.data)) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - }.filter(_.flatMap(_.topicId).isDefined) - .map(_.get) - .map(topicIdAssignment => (topicIdAssignment.topic, topicIdAssignment.topicId.get)) - .toMap + throw new UnsupportedOperationException() } /** @@ -700,7 +359,7 @@ class KafkaZkClient private[zk] ( * @return the replica assignment for each partition from the given topics. */ def getReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, Seq[Int]] = { - getFullReplicaAssignmentForTopics(topics).map { case (k, v) => k -> v.replicas } + throw new UnsupportedOperationException() } /** @@ -709,16 +368,7 @@ class KafkaZkClient private[zk] ( * @return the TopicIdReplicaAssignment for each partition for the given topics. */ def getReplicaAssignmentAndTopicIdForTopics(topics: Set[String]): Set[TopicIdReplicaAssignment] = { - val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic))) - val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq) - getDataResponses.map { getDataResponse => - val topic = getDataResponse.ctx.get.asInstanceOf[String] - getDataResponse.resultCode match { - case Code.OK => TopicZNode.decode(topic, getDataResponse.data) - case Code.NONODE => TopicIdReplicaAssignment(topic, None, Map.empty[TopicPartition, ReplicaAssignment]) - case _ => throw getDataResponse.resultException.get - } - }.toSet + throw new UnsupportedOperationException() } /** @@ -727,16 +377,7 @@ class KafkaZkClient private[zk] ( * @return the full replica assignment for each partition from the given topics. */ def getFullReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, ReplicaAssignment] = { - val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic))) - val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq) - getDataResponses.flatMap { getDataResponse => - val topic = getDataResponse.ctx.get.asInstanceOf[String] - getDataResponse.resultCode match { - case Code.OK => TopicZNode.decode(topic, getDataResponse.data).assignment - case Code.NONODE => Map.empty[TopicPartition, ReplicaAssignment] - case _ => throw getDataResponse.resultException.get - } - }.toMap + throw new UnsupportedOperationException() } /** @@ -745,18 +386,7 @@ class KafkaZkClient private[zk] ( * @return the partition assignment for each partition from the given topics. */ def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, ReplicaAssignment]] = { - val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic))) - val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq) - getDataResponses.flatMap { getDataResponse => - val topic = getDataResponse.ctx.get.asInstanceOf[String] - getDataResponse.resultCode match { - case Code.OK => - val partitionMap = TopicZNode.decode(topic, getDataResponse.data).assignment.map { case (k, v) => (k.partition, v) } - Map(topic -> partitionMap) - case Code.NONODE => Map.empty[String, Map[Int, ReplicaAssignment]] - case _ => throw getDataResponse.resultException.get - } - }.toMap + throw new UnsupportedOperationException() } /** @@ -765,11 +395,7 @@ class KafkaZkClient private[zk] ( * @return the partition array for each topic from the given topics. */ def getPartitionsForTopics(topics: Set[String]): Map[String, Seq[Int]] = { - getPartitionAssignmentForTopics(topics).map { topicAndPartitionMap => - val topic = topicAndPartitionMap._1 - val partitionMap = topicAndPartitionMap._2 - topic -> partitionMap.keys.toSeq.sorted - } + throw new UnsupportedOperationException() } /** @@ -778,11 +404,7 @@ class KafkaZkClient private[zk] ( * @return optional integer that is Some if the topic exists and None otherwise. */ def getTopicPartitionCount(topic: String): Option[Int] = { - val topicData = getReplicaAssignmentForTopics(Set(topic)) - if (topicData.nonEmpty) - Some(topicData.size) - else - None + throw new UnsupportedOperationException() } /** @@ -791,8 +413,7 @@ class KafkaZkClient private[zk] ( * @return List of assigned replicas */ def getReplicasForPartition(topicPartition: TopicPartition): Seq[Int] = { - val topicData = getReplicaAssignmentForTopics(Set(topicPartition.topic)) - topicData.getOrElse(topicPartition, Seq.empty) + throw new UnsupportedOperationException() } /** @@ -800,14 +421,7 @@ class KafkaZkClient private[zk] ( * @return all partitions in the cluster */ def getAllPartitions: Set[TopicPartition] = { - val topics = getChildren(TopicsZNode.path) - if (topics == null) Set.empty - else { - topics.flatMap { topic => - // The partitions path may not exist if the topic is in the process of being deleted - getChildren(TopicPartitionsZNode.path(topic)).map(_.toInt).map(new TopicPartition(topic, _)) - }.toSet - } + throw new UnsupportedOperationException() } /** @@ -818,11 +432,7 @@ class KafkaZkClient private[zk] ( * returns (None, ZkVersion.UnknownVersion) if node doesn't exist and throws exception for any error */ def getDataAndVersion(path: String): (Option[Array[Byte]], Int) = { - val (data, stat) = getDataAndStat(path) - stat match { - case ZkStat.NoStat => (data, ZkVersion.UnknownVersion) - case _ => (data, stat.getVersion) - } + throw new UnsupportedOperationException() } /** @@ -833,14 +443,7 @@ class KafkaZkClient private[zk] ( * returns (None, ZkStat.NoStat) if node doesn't exists and throws exception for any error */ def getDataAndStat(path: String): (Option[Array[Byte]], Stat) = { - val getDataRequest = GetDataRequest(path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - - getDataResponse.resultCode match { - case Code.OK => (Option(getDataResponse.data), getDataResponse.stat) - case Code.NONODE => (None, ZkStat.NoStat) - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -849,12 +452,7 @@ class KafkaZkClient private[zk] ( * @return list of child node names */ def getChildren(path : String): Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path, registerWatch = true)) - getChildrenResponse.resultCode match { - case Code.OK => getChildrenResponse.children - case Code.NONODE => Seq.empty - case _ => throw getChildrenResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -867,36 +465,7 @@ class KafkaZkClient private[zk] ( */ def conditionalUpdatePath(path: String, data: Array[Byte], expectVersion: Int, optionalChecker: Option[(KafkaZkClient, String, Array[Byte]) => (Boolean,Int)] = None): (Boolean, Int) = { - - val setDataRequest = SetDataRequest(path, data, expectVersion) - val setDataResponse = retryRequestUntilConnected(setDataRequest) - - setDataResponse.resultCode match { - case Code.OK => - debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d" - .format(path, Utils.utf8(data), expectVersion, setDataResponse.stat.getVersion)) - (true, setDataResponse.stat.getVersion) - - case Code.BADVERSION => - optionalChecker match { - case Some(checker) => checker(this, path, data) - case _ => - debug("Checker method is not passed skipping zkData match") - debug("Conditional update of path %s with data %s and expected version %d failed due to %s" - .format(path, Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage)) - (false, ZkVersion.UnknownVersion) - } - - case Code.NONODE => - debug("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, - Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage)) - (false, ZkVersion.UnknownVersion) - - case _ => - debug("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, - Utils.utf8(data), expectVersion, setDataResponse.resultException.get.getMessage)) - throw setDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -905,7 +474,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while setting or creating the znode */ def createDeleteTopicPath(topicName: String): Unit = { - createRecursive(DeleteTopicsTopicZNode.path(topicName)) + throw new UnsupportedOperationException() } /** @@ -914,7 +483,7 @@ class KafkaZkClient private[zk] ( * @return true if topic is marked for deletion, else false */ def isTopicMarkedForDeletion(topic: String): Boolean = { - pathExists(DeleteTopicsTopicZNode.path(topic)) + throw new UnsupportedOperationException() } /** @@ -922,12 +491,7 @@ class KafkaZkClient private[zk] ( * @return sequence of topics marked for deletion. */ def getTopicDeletions: Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(DeleteTopicsZNode.path, registerWatch = true)) - getChildrenResponse.resultCode match { - case Code.OK => getChildrenResponse.children - case Code.NONODE => Seq.empty - case _ => throw getChildrenResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -936,8 +500,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteTopicDeletions(topics: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - val deleteRequests = topics.map(topic => DeleteRequest(DeleteTopicsTopicZNode.path(topic), ZkVersion.MatchAnyVersion)) - retryRequestsUntilConnected(deleteRequests, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -945,19 +508,7 @@ class KafkaZkClient private[zk] ( * @return the reassignments for each partition. */ def getPartitionReassignment: collection.Map[TopicPartition, Seq[Int]] = { - val getDataRequest = GetDataRequest(ReassignPartitionsZNode.path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => - ReassignPartitionsZNode.decode(getDataResponse.data) match { - case Left(e) => - logger.warn(s"Ignoring partition reassignment due to invalid json: ${e.getMessage}", e) - Map.empty[TopicPartition, Seq[Int]] - case Right(assignments) => assignments - } - case Code.NONODE => Map.empty - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -971,26 +522,7 @@ class KafkaZkClient private[zk] ( */ @Deprecated def setOrCreatePartitionReassignment(reassignment: collection.Map[TopicPartition, Seq[Int]], expectedControllerEpochZkVersion: Int): Unit = { - - def set(reassignmentData: Array[Byte]): SetDataResponse = { - val setDataRequest = SetDataRequest(ReassignPartitionsZNode.path, reassignmentData, ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(setDataRequest, expectedControllerEpochZkVersion) - } - - def create(reassignmentData: Array[Byte]): CreateResponse = { - val createRequest = CreateRequest(ReassignPartitionsZNode.path, reassignmentData, defaultAcls(ReassignPartitionsZNode.path), - CreateMode.PERSISTENT) - retryRequestUntilConnected(createRequest, expectedControllerEpochZkVersion) - } - - val reassignmentData = ReassignPartitionsZNode.encode(reassignment) - val setDataResponse = set(reassignmentData) - setDataResponse.resultCode match { - case Code.NONODE => - val createDataResponse = create(reassignmentData) - createDataResponse.maybeThrow() - case _ => setDataResponse.maybeThrow() - } + throw new UnsupportedOperationException() } /** @@ -999,7 +531,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while creating the znode. */ def createPartitionReassignment(reassignment: Map[TopicPartition, Seq[Int]]): Unit = { - createRecursive(ReassignPartitionsZNode.path, ReassignPartitionsZNode.encode(reassignment)) + throw new UnsupportedOperationException() } /** @@ -1007,7 +539,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deletePartitionReassignment(expectedControllerEpochZkVersion: Int): Unit = { - deletePath(ReassignPartitionsZNode.path, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -1015,7 +547,7 @@ class KafkaZkClient private[zk] ( * @return true if reassign partitions is in progress, else false. */ def reassignPartitionsInProgress: Boolean = { - pathExists(ReassignPartitionsZNode.path) + throw new UnsupportedOperationException() } /** @@ -1024,15 +556,7 @@ class KafkaZkClient private[zk] ( * @return map containing LeaderIsrAndControllerEpoch of each partition for we were able to lookup the partition state. */ def getTopicPartitionStates(partitions: Seq[TopicPartition]): Map[TopicPartition, LeaderIsrAndControllerEpoch] = { - val getDataResponses = getTopicPartitionStatesRaw(partitions) - getDataResponses.flatMap { getDataResponse => - val partition = getDataResponse.ctx.get.asInstanceOf[TopicPartition] - getDataResponse.resultCode match { - case Code.OK => TopicPartitionStateZNode.decode(getDataResponse.data, getDataResponse.stat).map(partition -> _) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - }.toMap + throw new UnsupportedOperationException() } /** @@ -1041,14 +565,7 @@ class KafkaZkClient private[zk] ( * @return LeaderIsrAndControllerEpoch of the partition state if exists, else None */ def getTopicPartitionState(partition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { - val getDataResponse = getTopicPartitionStatesRaw(Seq(partition)).head - if (getDataResponse.resultCode == Code.OK) { - TopicPartitionStateZNode.decode(getDataResponse.data, getDataResponse.stat) - } else if (getDataResponse.resultCode == Code.NONODE) { - None - } else { - throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1057,7 +574,7 @@ class KafkaZkClient private[zk] ( * @return optional integer if the leader exists and None otherwise. */ def getLeaderForPartition(partition: TopicPartition): Option[Int] = - getTopicPartitionState(partition).map(_.leaderAndIsr.leader) + throw new UnsupportedOperationException() /** * Gets the in-sync replicas (ISR) for a specific topicPartition @@ -1065,7 +582,7 @@ class KafkaZkClient private[zk] ( * @return optional ISR if exists and None otherwise */ def getInSyncReplicasForPartition(partition: TopicPartition): Option[Seq[Int]] = - getTopicPartitionState(partition).map(_.leaderAndIsr.isr.asScala.map(_.toInt)) + throw new UnsupportedOperationException() /** @@ -1074,7 +591,7 @@ class KafkaZkClient private[zk] ( * @return optional integer if the leader exists and None otherwise */ def getEpochForPartition(partition: TopicPartition): Option[Int] = { - getTopicPartitionState(partition).map(_.leaderAndIsr.leaderEpoch) + throw new UnsupportedOperationException() } /** @@ -1082,12 +599,7 @@ class KafkaZkClient private[zk] ( * @return sequence of znode names and not the absolute znode path. */ def getAllIsrChangeNotifications: Seq[String] = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(IsrChangeNotificationZNode.path, registerWatch = true)) - getChildrenResponse.resultCode match { - case Code.OK => getChildrenResponse.children.map(IsrChangeNotificationSequenceZNode.sequenceNumber) - case Code.NONODE => Seq.empty - case _ => throw getChildrenResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1096,17 +608,7 @@ class KafkaZkClient private[zk] ( * @return partitions associated with the given isr change notifications. */ def getPartitionsFromIsrChangeNotifications(sequenceNumbers: Seq[String]): Seq[TopicPartition] = { - val getDataRequests = sequenceNumbers.map { sequenceNumber => - GetDataRequest(IsrChangeNotificationSequenceZNode.path(sequenceNumber)) - } - val getDataResponses = retryRequestsUntilConnected(getDataRequests) - getDataResponses.flatMap { getDataResponse => - getDataResponse.resultCode match { - case Code.OK => IsrChangeNotificationSequenceZNode.decode(getDataResponse.data) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - } + throw new UnsupportedOperationException() } /** @@ -1114,12 +616,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteIsrChangeNotifications(expectedControllerEpochZkVersion: Int): Unit = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(IsrChangeNotificationZNode.path, registerWatch = true)) - if (getChildrenResponse.resultCode == Code.OK) { - deleteIsrChangeNotifications(getChildrenResponse.children.map(IsrChangeNotificationSequenceZNode.sequenceNumber), expectedControllerEpochZkVersion) - } else if (getChildrenResponse.resultCode != Code.NONODE) { - getChildrenResponse.maybeThrow() - } + throw new UnsupportedOperationException() } /** @@ -1128,10 +625,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteIsrChangeNotifications(sequenceNumbers: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - val deleteRequests = sequenceNumbers.map { sequenceNumber => - DeleteRequest(IsrChangeNotificationSequenceZNode.path(sequenceNumber), ZkVersion.MatchAnyVersion) - } - retryRequestsUntilConnected(deleteRequests, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -1140,7 +634,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while creating the znode */ def createPreferredReplicaElection(partitions: Set[TopicPartition]): Unit = { - createRecursive(PreferredReplicaElectionZNode.path, PreferredReplicaElectionZNode.encode(partitions)) + throw new UnsupportedOperationException() } /** @@ -1148,13 +642,7 @@ class KafkaZkClient private[zk] ( * @return sequence of partitions. */ def getPreferredReplicaElection: Set[TopicPartition] = { - val getDataRequest = GetDataRequest(PreferredReplicaElectionZNode.path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => PreferredReplicaElectionZNode.decode(getDataResponse.data) - case Code.NONODE => Set.empty - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1162,8 +650,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deletePreferredReplicaElection(expectedControllerEpochZkVersion: Int): Unit = { - val deleteRequest = DeleteRequest(PreferredReplicaElectionZNode.path, ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(deleteRequest, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -1171,24 +658,7 @@ class KafkaZkClient private[zk] ( * @return optional integer that is Some if the controller znode exists and can be parsed and None otherwise. */ def getControllerId: Option[Int] = { - val getDataRequest = GetDataRequest(ControllerZNode.path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => ControllerZNode.decode(getDataResponse.data) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } - } - - - private def getControllerRegistration: Option[ZKControllerRegistration] = { - val getDataRequest = GetDataRequest(ControllerZNode.path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => Some(ControllerZNode.decodeController(getDataResponse.data, getDataResponse.stat.getVersion)) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1196,8 +666,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteController(expectedControllerEpochZkVersion: Int): Unit = { - val deleteRequest = DeleteRequest(ControllerZNode.path, ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(deleteRequest, expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -1205,15 +674,7 @@ class KafkaZkClient private[zk] ( * @return optional (Int, Stat) that is Some if the controller epoch path exists and None otherwise. */ def getControllerEpoch: Option[(Int, Stat)] = { - val getDataRequest = GetDataRequest(ControllerEpochZNode.path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => - val epoch = ControllerEpochZNode.decode(getDataResponse.data) - Option(epoch, getDataResponse.stat) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1222,7 +683,7 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteTopicZNode(topic: String, expectedControllerEpochZkVersion: Int): Unit = { - deleteRecursive(TopicZNode.path(topic), expectedControllerEpochZkVersion) + throw new UnsupportedOperationException() } /** @@ -1231,195 +692,11 @@ class KafkaZkClient private[zk] ( * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. */ def deleteTopicConfigs(topics: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - val deleteRequests = topics.map(topic => DeleteRequest(ConfigEntityZNode.path(ConfigType.TOPIC, topic), - ZkVersion.MatchAnyVersion)) - retryRequestsUntilConnected(deleteRequests, expectedControllerEpochZkVersion) - } - - //Acl management methods - - /** - * Creates the required zk nodes for Acl storage and Acl change storage. - */ - def createAclPaths(): Unit = { - ZkAclStore.stores.foreach(store => { - createRecursive(store.aclPath, throwIfPathExists = false) - AclEntry.RESOURCE_TYPES.forEach(resourceType => createRecursive(store.path(resourceType), throwIfPathExists = false)) - }) - - ZkAclChangeStore.stores.foreach(store => createRecursive(store.aclChangePath, throwIfPathExists = false)) - } - - /** - * Gets VersionedAcls for a given Resource - * @param resource Resource to get VersionedAcls for - * @return VersionedAcls - */ - def getVersionedAclsForResource(resource: ResourcePattern): ZkData.VersionedAcls = { - val getDataRequest = GetDataRequest(ResourceZNode.path(resource)) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => ResourceZNode.decode(getDataResponse.data, getDataResponse.stat) - case Code.NONODE => ZkData.VersionedAcls(Set.empty, ZkVersion.UnknownVersion) - case _ => throw getDataResponse.resultException.get - } - } - - /** - * Sets or creates the resource znode path with the given acls and expected zk version depending - * on whether it already exists or not. - * @param resource - * @param aclsSet - * @param expectedVersion - * @return true if the update was successful and the new version - */ - def conditionalSetAclsForResource(resource: ResourcePattern, - aclsSet: Set[AclEntry], - expectedVersion: Int): (Boolean, Int) = { - def set(aclData: Array[Byte], expectedVersion: Int): SetDataResponse = { - val setDataRequest = SetDataRequest(ResourceZNode.path(resource), aclData, expectedVersion) - retryRequestUntilConnected(setDataRequest) - } - - if (expectedVersion < 0) - throw new IllegalArgumentException(s"Invalid version $expectedVersion provided for conditional update") - - val aclData = ResourceZNode.encode(aclsSet) - - val setDataResponse = set(aclData, expectedVersion) - setDataResponse.resultCode match { - case Code.OK => (true, setDataResponse.stat.getVersion) - case Code.NONODE | Code.BADVERSION => (false, ZkVersion.UnknownVersion) - case _ => throw setDataResponse.resultException.get - } - } - - def createAclsForResourceIfNotExists(resource: ResourcePattern, aclsSet: Set[AclEntry]): (Boolean, Int) = { - def create(aclData: Array[Byte]): CreateResponse = { - val path = ResourceZNode.path(resource) - val createRequest = CreateRequest(path, aclData, defaultAcls(path), CreateMode.PERSISTENT) - retryRequestUntilConnected(createRequest) - } - - val aclData = ResourceZNode.encode(aclsSet) - - val createResponse = create(aclData) - createResponse.resultCode match { - case Code.OK => (true, 0) - case Code.NODEEXISTS => (false, ZkVersion.UnknownVersion) - case _ => throw createResponse.resultException.get - } - } - - /** - * Creates an Acl change notification message. - * @param resource resource pattern that has changed - */ - def createAclChangeNotification(resource: ResourcePattern): Unit = { - val aclChange = ZkAclStore(resource.patternType).changeStore.createChangeNode(resource) - val createRequest = CreateRequest(aclChange.path, aclChange.bytes, defaultAcls(aclChange.path), CreateMode.PERSISTENT_SEQUENTIAL) - val createResponse = retryRequestUntilConnected(createRequest) - createResponse.maybeThrow() + throw new UnsupportedOperationException() } def propagateLogDirEvent(brokerId: Int): Unit = { - val logDirEventNotificationPath: String = createSequentialPersistentPath( - LogDirEventNotificationZNode.path + "/" + LogDirEventNotificationSequenceZNode.SequenceNumberPrefix, - LogDirEventNotificationSequenceZNode.encode(brokerId)) - debug(s"Added $logDirEventNotificationPath for broker $brokerId") - } - - def propagateIsrChanges(isrChangeSet: collection.Set[TopicPartition]): Unit = { - val isrChangeNotificationPath: String = createSequentialPersistentPath(IsrChangeNotificationSequenceZNode.path(), - IsrChangeNotificationSequenceZNode.encode(isrChangeSet)) - debug(s"Added $isrChangeNotificationPath for $isrChangeSet") - } - - /** - * Deletes all Acl change notifications. - * @throws KeeperException if there is an error while deleting Acl change notifications - */ - def deleteAclChangeNotifications(): Unit = { - ZkAclChangeStore.stores.foreach(store => { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(store.aclChangePath, registerWatch = true)) - if (getChildrenResponse.resultCode == Code.OK) { - deleteAclChangeNotifications(store.aclChangePath, getChildrenResponse.children) - } else if (getChildrenResponse.resultCode != Code.NONODE) { - getChildrenResponse.maybeThrow() - } - }) - } - - /** - * Deletes the Acl change notifications associated with the given sequence nodes - * - * @param aclChangePath the root path - * @param sequenceNodes the name of the node to delete. - */ - private def deleteAclChangeNotifications(aclChangePath: String, sequenceNodes: Seq[String]): Unit = { - val deleteRequests = sequenceNodes.map { sequenceNode => - DeleteRequest(s"$aclChangePath/$sequenceNode", ZkVersion.MatchAnyVersion) - } - - val deleteResponses = retryRequestsUntilConnected(deleteRequests) - deleteResponses.foreach { deleteResponse => - if (deleteResponse.resultCode != Code.NONODE) { - deleteResponse.maybeThrow() - } - } - } - - /** - * Gets the resource types, for which ACLs are stored, for the supplied resource pattern type. - * @param patternType The resource pattern type to retrieve the names for. - * @return list of resource type names - */ - def getResourceTypes(patternType: PatternType): Seq[String] = { - getChildren(ZkAclStore(patternType).aclPath) - } - - /** - * Gets the resource names, for which ACLs are stored, for a given resource type and pattern type - * @param patternType The resource pattern type to retrieve the names for. - * @param resourceType Resource type to retrieve the names for. - * @return list of resource names - */ - def getResourceNames(patternType: PatternType, resourceType: ResourceType): Seq[String] = { - getChildren(ZkAclStore(patternType).path(resourceType)) - } - - /** - * Deletes the given Resource node - * @param resource - * @return delete status - */ - def deleteResource(resource: ResourcePattern): Boolean = { - deleteRecursive(ResourceZNode.path(resource)) - } - - /** - * checks the resource existence - * @param resource - * @return existence status - */ - def resourceExists(resource: ResourcePattern): Boolean = { - pathExists(ResourceZNode.path(resource)) - } - - /** - * Conditional delete the resource node - * @param resource - * @param expectedVersion - * @return return true if it succeeds, false otherwise (the current version is not the expected version) - */ - def conditionalDelete(resource: ResourcePattern, expectedVersion: Int): Boolean = { - val deleteRequest = DeleteRequest(ResourceZNode.path(resource), expectedVersion) - val deleteResponse = retryRequestUntilConnected(deleteRequest) - deleteResponse.resultCode match { - case Code.OK | Code.NONODE => true - case Code.BADVERSION => false - case _ => throw deleteResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1430,23 +707,14 @@ class KafkaZkClient private[zk] ( * @return KeeperException if there is an error while deleting the path */ def deletePath(path: String, expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion, recursiveDelete: Boolean = true): Unit = { - if (recursiveDelete) - deleteRecursive(path, expectedControllerEpochZkVersion) - else { - val deleteRequest = DeleteRequest(path, ZkVersion.MatchAnyVersion) - val deleteResponse = retryRequestUntilConnected(deleteRequest, expectedControllerEpochZkVersion) - if (deleteResponse.resultCode != Code.OK && deleteResponse.resultCode != Code.NONODE) { - throw deleteResponse.resultException.get - } - } + throw new UnsupportedOperationException() } /** * Creates the required zk nodes for Delegation Token storage */ def createDelegationTokenPaths(): Unit = { - createRecursive(DelegationTokenChangeNotificationZNode.path, throwIfPathExists = false) - createRecursive(DelegationTokensZNode.path, throwIfPathExists = false) + throw new UnsupportedOperationException() } /** @@ -1454,10 +722,7 @@ class KafkaZkClient private[zk] ( * @param tokenId token Id */ def createTokenChangeNotification(tokenId: String): Unit = { - val path = DelegationTokenChangeNotificationSequenceZNode.createPath - val createRequest = CreateRequest(path, DelegationTokenChangeNotificationSequenceZNode.encode(tokenId), defaultAcls(path), CreateMode.PERSISTENT_SEQUENTIAL) - val createResponse = retryRequestUntilConnected(createRequest) - createResponse.resultException.foreach(e => throw e) + throw new UnsupportedOperationException() } /** @@ -1468,26 +733,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while setting or creating the znode */ def setOrCreateDelegationToken(token: DelegationToken): Unit = { - - def set(tokenData: Array[Byte]): SetDataResponse = { - val setDataRequest = SetDataRequest(DelegationTokenInfoZNode.path(token.tokenInfo().tokenId()), tokenData, ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(setDataRequest) - } - - def create(tokenData: Array[Byte]): CreateResponse = { - val path = DelegationTokenInfoZNode.path(token.tokenInfo().tokenId()) - val createRequest = CreateRequest(path, tokenData, defaultAcls(path), CreateMode.PERSISTENT) - retryRequestUntilConnected(createRequest) - } - - val tokenInfo = DelegationTokenInfoZNode.encode(token.tokenInfo()) - val setDataResponse = set(tokenInfo) - setDataResponse.resultCode match { - case Code.NONODE => - val createDataResponse = create(tokenInfo) - createDataResponse.maybeThrow() - case _ => setDataResponse.maybeThrow() - } + throw new UnsupportedOperationException() } /** @@ -1495,13 +741,7 @@ class KafkaZkClient private[zk] ( * @return optional TokenInfo that is Some if the token znode exists and can be parsed and None otherwise. */ def getDelegationTokenInfo(delegationTokenId: String): Option[TokenInformation] = { - val getDataRequest = GetDataRequest(DelegationTokenInfoZNode.path(delegationTokenId)) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => DelegationTokenInfoZNode.decode(getDataResponse.data) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1510,7 +750,7 @@ class KafkaZkClient private[zk] ( * @return delete status */ def deleteDelegationToken(delegationTokenId: String): Boolean = { - deleteRecursive(DelegationTokenInfoZNode.path(delegationTokenId)) + throw new UnsupportedOperationException() } /** @@ -1522,13 +762,7 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if an error is returned by ZooKeeper */ def registerZNodeChangeHandlerAndCheckExistence(zNodeChangeHandler: ZNodeChangeHandler): Boolean = { - zooKeeperClient.registerZNodeChangeHandler(zNodeChangeHandler) - val existsResponse = retryRequestUntilConnected(ExistsRequest(zNodeChangeHandler.path)) - existsResponse.resultCode match { - case Code.OK => true - case Code.NONODE => false - case _ => throw existsResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1536,7 +770,7 @@ class KafkaZkClient private[zk] ( * @param zNodeChangeHandler */ def registerZNodeChangeHandler(zNodeChangeHandler: ZNodeChangeHandler): Unit = { - zooKeeperClient.registerZNodeChangeHandler(zNodeChangeHandler) + throw new UnsupportedOperationException() } /** @@ -1544,7 +778,7 @@ class KafkaZkClient private[zk] ( * @param path */ def unregisterZNodeChangeHandler(path: String): Unit = { - zooKeeperClient.unregisterZNodeChangeHandler(path) + throw new UnsupportedOperationException() } /** @@ -1552,7 +786,7 @@ class KafkaZkClient private[zk] ( * @param zNodeChildChangeHandler */ def registerZNodeChildChangeHandler(zNodeChildChangeHandler: ZNodeChildChangeHandler): Unit = { - zooKeeperClient.registerZNodeChildChangeHandler(zNodeChildChangeHandler) + throw new UnsupportedOperationException() } /** @@ -1560,7 +794,7 @@ class KafkaZkClient private[zk] ( * @param path */ def unregisterZNodeChildChangeHandler(path: String): Unit = { - zooKeeperClient.unregisterZNodeChildChangeHandler(path) + throw new UnsupportedOperationException() } /** @@ -1568,7 +802,7 @@ class KafkaZkClient private[zk] ( * @param stateChangeHandler */ def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { - zooKeeperClient.registerStateChangeHandler(stateChangeHandler) + throw new UnsupportedOperationException() } /** @@ -1576,15 +810,14 @@ class KafkaZkClient private[zk] ( * @param name */ def unregisterStateChangeHandler(name: String): Unit = { - zooKeeperClient.unregisterStateChangeHandler(name) + throw new UnsupportedOperationException() } /** * Close the underlying ZooKeeperClient. */ def close(): Unit = { - metricsGroup.removeMetric("ZooKeeperRequestLatencyMs") - zooKeeperClient.close() + throw new UnsupportedOperationException() } /** @@ -1594,13 +827,7 @@ class KafkaZkClient private[zk] ( * @return optional long that is Some if there was an offset committed for topic partition, group and None otherwise. */ def getConsumerOffset(group: String, topicPartition: TopicPartition): Option[Long] = { - val getDataRequest = GetDataRequest(ConsumerOffset.path(group, topicPartition.topic, topicPartition.partition)) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => ConsumerOffset.decode(getDataResponse.data) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1610,12 +837,7 @@ class KafkaZkClient private[zk] ( * @param offset the offset value */ def setOrCreateConsumerOffset(group: String, topicPartition: TopicPartition, offset: Long): Unit = { - val setDataResponse = setConsumerOffset(group, topicPartition, offset) - if (setDataResponse.resultCode == Code.NONODE) { - createConsumerOffset(group, topicPartition, offset) - } else { - setDataResponse.maybeThrow() - } + throw new UnsupportedOperationException() } /** @@ -1623,13 +845,7 @@ class KafkaZkClient private[zk] ( * @return optional cluster id in String. */ def getClusterId: Option[String] = { - val getDataRequest = GetDataRequest(ClusterIdZNode.path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - getDataResponse.resultCode match { - case Code.OK => Some(ClusterIdZNode.fromJson(getDataResponse.data)) - case Code.NONODE => None - case _ => throw getDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1638,12 +854,7 @@ class KafkaZkClient private[zk] ( * @return the ACL array of the given node. */ def getAcl(path: String): Seq[ACL] = { - val getAclRequest = GetAclRequest(path) - val getAclResponse = retryRequestUntilConnected(getAclRequest) - getAclResponse.resultCode match { - case Code.OK => getAclResponse.acl - case _ => throw getAclResponse.resultException.get - } + throw new UnsupportedOperationException() } /** @@ -1652,9 +863,7 @@ class KafkaZkClient private[zk] ( * @param acl the given acl for the node */ def setAcl(path: String, acl: Seq[ACL]): Unit = { - val setAclRequest = SetAclRequest(path, acl, ZkVersion.MatchAnyVersion) - val setAclResponse = retryRequestUntilConnected(setAclRequest) - setAclResponse.maybeThrow() + throw new UnsupportedOperationException() } /** @@ -1662,13 +871,7 @@ class KafkaZkClient private[zk] ( * @return cluster id */ def createOrGetClusterId(proposedClusterId: String): String = { - try { - createRecursive(ClusterIdZNode.path, ClusterIdZNode.toJson(proposedClusterId)) - proposedClusterId - } catch { - case _: NodeExistsException => getClusterId.getOrElse( - throw new KafkaException("Failed to get cluster id from Zookeeper. This can happen if /cluster/id is deleted from Zookeeper.")) - } + throw new UnsupportedOperationException() } /** @@ -1677,23 +880,14 @@ class KafkaZkClient private[zk] ( * @return sequence number as the broker id */ def generateBrokerSequenceId(): Int = { - val setDataRequest = SetDataRequest(BrokerSequenceIdZNode.path, Array.empty[Byte], ZkVersion.MatchAnyVersion) - val setDataResponse = retryRequestUntilConnected(setDataRequest) - setDataResponse.resultCode match { - case Code.OK => setDataResponse.stat.getVersion - case Code.NONODE => - // maker sure the path exists - createRecursive(BrokerSequenceIdZNode.path, Array.empty[Byte], throwIfPathExists = false) - generateBrokerSequenceId() - case _ => throw setDataResponse.resultException.get - } + throw new UnsupportedOperationException() } /** * Pre-create top level paths in ZK if needed. */ def createTopLevelPaths(): Unit = { - ZkData.PersistentZkPaths.foreach(makeSurePersistentPathExists) + throw new UnsupportedOperationException() } /** @@ -1701,42 +895,15 @@ class KafkaZkClient private[zk] ( * @param path */ def makeSurePersistentPathExists(path: String): Unit = { - createRecursive(path, data = null, throwIfPathExists = false) + throw new UnsupportedOperationException() } def createFeatureZNode(nodeContents: FeatureZNode): Unit = { - val createRequest = CreateRequest( - FeatureZNode.path, - FeatureZNode.encode(nodeContents), - defaultAcls(FeatureZNode.path), - CreateMode.PERSISTENT) - val response = retryRequestUntilConnected(createRequest) - response.maybeThrow() + throw new UnsupportedOperationException() } def updateFeatureZNode(nodeContents: FeatureZNode): Int = { - val setRequest = SetDataRequest( - FeatureZNode.path, - FeatureZNode.encode(nodeContents), - ZkVersion.MatchAnyVersion) - val response = retryRequestUntilConnected(setRequest) - response.maybeThrow() - response.stat.getVersion - } - - def deleteFeatureZNode(): Unit = { - deletePath(FeatureZNode.path, ZkVersion.MatchAnyVersion, recursiveDelete = false) - } - - private def setConsumerOffset(group: String, topicPartition: TopicPartition, offset: Long): SetDataResponse = { - val setDataRequest = SetDataRequest(ConsumerOffset.path(group, topicPartition.topic, topicPartition.partition), - ConsumerOffset.encode(offset), ZkVersion.MatchAnyVersion) - retryRequestUntilConnected(setDataRequest) - } - - private def createConsumerOffset(group: String, topicPartition: TopicPartition, offset: Long): Unit = { - val path = ConsumerOffset.path(group, topicPartition.topic, topicPartition.partition) - createRecursive(path, ConsumerOffset.encode(offset)) + throw new UnsupportedOperationException() } /** @@ -1747,253 +914,19 @@ class KafkaZkClient private[zk] ( * @throws KeeperException if there is an error while deleting the znodes */ def deleteRecursive(path: String, expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion): Boolean = { - val getChildrenResponse = retryRequestUntilConnected(GetChildrenRequest(path, registerWatch = true)) - getChildrenResponse.resultCode match { - case Code.OK => - getChildrenResponse.children.foreach(child => deleteRecursive(s"$path/$child", expectedControllerEpochZkVersion)) - val deleteResponse = retryRequestUntilConnected(DeleteRequest(path, ZkVersion.MatchAnyVersion), expectedControllerEpochZkVersion) - if (deleteResponse.resultCode != Code.OK && deleteResponse.resultCode != Code.NONODE) - throw deleteResponse.resultException.get - true - case Code.NONODE => false - case _ => throw getChildrenResponse.resultException.get - } + throw new UnsupportedOperationException() } def pathExists(path: String): Boolean = { - val existsRequest = ExistsRequest(path) - val existsResponse = retryRequestUntilConnected(existsRequest) - existsResponse.resultCode match { - case Code.OK => true - case Code.NONODE => false - case _ => throw existsResponse.resultException.get - } + throw new UnsupportedOperationException() } - private[kafka] def createRecursive(path: String, data: Array[Byte] = null, throwIfPathExists: Boolean = true): Unit = { - - def parentPath(path: String): String = { - val indexOfLastSlash = path.lastIndexOf("/") - if (indexOfLastSlash == -1) throw new IllegalArgumentException(s"Invalid path $path") - path.substring(0, indexOfLastSlash) - } - - def createRecursive0(path: String): Unit = { - val createRequest = CreateRequest(path, null, defaultAcls(path), CreateMode.PERSISTENT) - var createResponse = retryRequestUntilConnected(createRequest) - if (createResponse.resultCode == Code.NONODE) { - createRecursive0(parentPath(path)) - createResponse = retryRequestUntilConnected(createRequest) - if (createResponse.resultCode != Code.OK && createResponse.resultCode != Code.NODEEXISTS) { - throw createResponse.resultException.get - } - } else if (createResponse.resultCode != Code.OK && createResponse.resultCode != Code.NODEEXISTS) { - throw createResponse.resultException.get - } - } - - val createRequest = CreateRequest(path, data, defaultAcls(path), CreateMode.PERSISTENT) - var createResponse = retryRequestUntilConnected(createRequest) - - if (throwIfPathExists && createResponse.resultCode == Code.NODEEXISTS) { - createResponse.maybeThrow() - } else if (createResponse.resultCode == Code.NONODE) { - createRecursive0(parentPath(path)) - createResponse = retryRequestUntilConnected(createRequest) - if (throwIfPathExists || createResponse.resultCode != Code.NODEEXISTS) - createResponse.maybeThrow() - } else if (createResponse.resultCode != Code.NODEEXISTS) - createResponse.maybeThrow() - - } - - private def createTopicPartition(partitions: Seq[TopicPartition], expectedControllerEpochZkVersion: Int): Seq[CreateResponse] = { - val createRequests = partitions.map { partition => - val path = TopicPartitionZNode.path(partition) - CreateRequest(path, null, defaultAcls(path), CreateMode.PERSISTENT, Some(partition)) - } - retryRequestsUntilConnected(createRequests, expectedControllerEpochZkVersion) - } - - private def createTopicPartitions(topics: Seq[String], expectedControllerEpochZkVersion: Int): Seq[CreateResponse] = { - val createRequests = topics.map { topic => - val path = TopicPartitionsZNode.path(topic) - CreateRequest(path, null, defaultAcls(path), CreateMode.PERSISTENT, Some(topic)) - } - retryRequestsUntilConnected(createRequests, expectedControllerEpochZkVersion) - } - - private def getTopicConfigs(topics: Set[String]): Seq[GetDataResponse] = { - val getDataRequests: Seq[GetDataRequest] = topics.iterator.map { topic => - GetDataRequest(ConfigEntityZNode.path(ConfigType.TOPIC, topic), ctx = Some(topic)) - }.toBuffer - - retryRequestsUntilConnected(getDataRequests) - } - - def defaultAcls(path: String): Seq[ACL] = ZkData.defaultAcls(isSecure, path) - - def secure: Boolean = isSecure - - private[zk] def retryRequestUntilConnected[Req <: AsyncRequest](request: Req, expectedControllerZkVersion: Int = ZkVersion.MatchAnyVersion): Req#Response = { - retryRequestsUntilConnected(Seq(request), expectedControllerZkVersion).head - } - - private def retryRequestsUntilConnected[Req <: AsyncRequest](requests: Seq[Req], expectedControllerZkVersion: Int): Seq[Req#Response] = { - expectedControllerZkVersion match { - case ZkVersion.MatchAnyVersion => retryRequestsUntilConnected(requests) - case version if version >= 0 => - retryRequestsUntilConnected(requests.map(wrapRequestWithControllerEpochCheck(_, version))) - .map(unwrapResponseWithControllerEpochCheck(_).asInstanceOf[Req#Response]) - case invalidVersion => - throw new IllegalArgumentException(s"Expected controller epoch zkVersion $invalidVersion should be non-negative or equal to ${ZkVersion.MatchAnyVersion}") - } - } - - private def retryRequestsUntilConnected[Req <: AsyncRequest](requests: Seq[Req]): Seq[Req#Response] = { - val remainingRequests = new mutable.ArrayBuffer(requests.size) ++= requests - val responses = new mutable.ArrayBuffer[Req#Response] - while (remainingRequests.nonEmpty) { - val batchResponses = zooKeeperClient.handleRequests(remainingRequests) - - batchResponses.foreach(response => latencyMetric.update(response.metadata.responseTimeMs)) - - // Only execute slow path if we find a response with CONNECTIONLOSS - if (batchResponses.exists(_.resultCode == Code.CONNECTIONLOSS)) { - val requestResponsePairs = remainingRequests.zip(batchResponses) - - remainingRequests.clear() - requestResponsePairs.foreach { case (request, response) => - if (response.resultCode == Code.CONNECTIONLOSS) - remainingRequests += request - else - responses += response - } - - if (remainingRequests.nonEmpty) - zooKeeperClient.waitUntilConnected() - } else { - remainingRequests.clear() - responses ++= batchResponses - } - } - responses - } - - private def checkedEphemeralCreate(path: String, data: Array[Byte]): Stat = { - val checkedEphemeral = new CheckedEphemeral(path, data) - info(s"Creating $path (is it secure? $isSecure)") - val stat = checkedEphemeral.create() - info(s"Stat of the created znode at $path is: $stat") - stat - } - - private def isZKSessionIdDiffFromCurrentZKSessionId: Boolean = { - zooKeeperClient.sessionId != currentZooKeeperSessionId - } - - private def isZKSessionTheEphemeralOwner(ephemeralOwnerId: Long): Boolean = { - ephemeralOwnerId == currentZooKeeperSessionId - } - - private[zk] def shouldReCreateEphemeralZNode(ephemeralOwnerId: Long): Boolean = { - isZKSessionTheEphemeralOwner(ephemeralOwnerId) && isZKSessionIdDiffFromCurrentZKSessionId - } - - private def updateCurrentZKSessionId(newSessionId: Long): Unit = { - currentZooKeeperSessionId = newSessionId - } - - private class CheckedEphemeral(path: String, data: Array[Byte]) extends Logging { - def create(): Stat = { - val response = retryRequestUntilConnected( - MultiRequest(Seq( - CreateOp(path, null, defaultAcls(path), CreateMode.EPHEMERAL), - SetDataOp(path, data, 0))) - ) - val stat = response.resultCode match { - case Code.OK => - val setDataResult = response.zkOpResults(1).rawOpResult.asInstanceOf[SetDataResult] - setDataResult.getStat - case Code.NODEEXISTS => - getAfterNodeExists - case code => - error(s"Error while creating ephemeral at $path with return code: $code") - throw KeeperException.create(code) - } - - // At this point, we need to save a reference to the zookeeper session id. - // This is done here since the Zookeeper session id may not be available at the Object creation time. - // This is assuming the 'retryRequestUntilConnected' method got connected and a valid session id is present. - // This code is part of the workaround done in the KAFKA-7165, once ZOOKEEPER-2985 is complete, this code - // must be deleted. - updateCurrentZKSessionId(zooKeeperClient.sessionId) - - stat - } - - // This method is part of the work around done in the KAFKA-7165, once ZOOKEEPER-2985 is complete, this code must - // be deleted. - private def delete(): Code = { - val deleteRequest = DeleteRequest(path, ZkVersion.MatchAnyVersion) - val deleteResponse = retryRequestUntilConnected(deleteRequest) - deleteResponse.resultCode match { - case code@ Code.OK => code - case code@ Code.NONODE => code - case code => - error(s"Error while deleting ephemeral node at $path with return code: $code") - code - } - } - - private def reCreate(): Stat = { - val codeAfterDelete = delete() - val codeAfterReCreate = codeAfterDelete - debug(s"Result of znode ephemeral deletion at $path is: $codeAfterDelete") - if (codeAfterDelete == Code.OK || codeAfterDelete == Code.NONODE) { - create() - } else { - throw KeeperException.create(codeAfterReCreate) - } - } - - private def getAfterNodeExists: Stat = { - val getDataRequest = GetDataRequest(path) - val getDataResponse = retryRequestUntilConnected(getDataRequest) - val ephemeralOwnerId = getDataResponse.stat.getEphemeralOwner - getDataResponse.resultCode match { - // At this point, the Zookeeper session could be different (due a 'Session expired') from the one that initially - // registered the Broker into the Zookeeper ephemeral node, but the znode is still present in ZooKeeper. - // The expected behaviour is that Zookeeper server removes the ephemeral node associated with the expired session - // but due an already reported bug in Zookeeper (ZOOKEEPER-2985) this is not happening, so, the following check - // will validate if this Broker got registered with the previous (expired) session and try to register again, - // deleting the ephemeral node and creating it again. - // This code is part of the work around done in the KAFKA-7165, once ZOOKEEPER-2985 is complete, this code must - // be deleted. - case Code.OK if shouldReCreateEphemeralZNode(ephemeralOwnerId) => - info(s"Was not possible to create the ephemeral at $path, node already exists and owner " + - s"'0x${JLong.toHexString(ephemeralOwnerId)}' does not match current session '0x${JLong.toHexString(zooKeeperClient.sessionId)}'" + - s", trying to delete and re-create it with the newest Zookeeper session") - reCreate() - case Code.OK if ephemeralOwnerId != zooKeeperClient.sessionId => - error(s"Error while creating ephemeral at $path, node already exists and owner " + - s"'0x${JLong.toHexString(ephemeralOwnerId)}' does not match current session '0x${JLong.toHexString(zooKeeperClient.sessionId)}'") - throw KeeperException.create(Code.NODEEXISTS) - case Code.OK => - getDataResponse.stat - case Code.NONODE => - info(s"The ephemeral node at $path went away while reading it, attempting create() again") - create() - case code => - error(s"Error while creating ephemeral at $path as it already exists and error getting the node data due to $code") - throw KeeperException.create(code) - } - } + def defaultAcls(path: String): Seq[ACL] = { + throw new UnsupportedOperationException() } } object KafkaZkClient { - /** * @param finishedPartitions Partitions that finished either in successfully * updated partition states or failed with an exception. @@ -2005,149 +938,4 @@ object KafkaZkClient { finishedPartitions: Map[TopicPartition, Either[Exception, LeaderAndIsr]], partitionsToRetry: Seq[TopicPartition] ) - - /** - * Create an instance of this class with the provided parameters. - * - * The metric group and type are preserved by default for compatibility with previous versions. - */ - def apply(connectString: String, - isSecure: Boolean, - sessionTimeoutMs: Int, - connectionTimeoutMs: Int, - maxInFlightRequests: Int, - time: Time, - name: String, - zkClientConfig: ZKClientConfig, - metricGroup: String = "kafka.server", - metricType: String = "SessionExpireListener", - createChrootIfNecessary: Boolean = false, - enableEntityConfigControllerCheck: Boolean = true - ): KafkaZkClient = { - - /* ZooKeeper 3.6.0 changed the default configuration for JUTE_MAXBUFFER from 4 MB to 1 MB. - * This causes a regression if Kafka tries to retrieve a large amount of data across many - * znodes – in such a case the ZooKeeper client will repeatedly emit a message of the form - * "java.io.IOException: Packet len <####> is out of range". - * - * We restore the 3.4.x/3.5.x behavior unless the caller has set the property (note that ZKConfig - * auto configures itself if certain system properties have been set). - * - * See https://github.com/apache/zookeeper/pull/1129 for the details on why the behavior - * changed in 3.6.0. - */ - if (zkClientConfig.getProperty(ZKConfig.JUTE_MAXBUFFER) == null) - zkClientConfig.setProperty(ZKConfig.JUTE_MAXBUFFER, (4096 * 1024).toString) - - if (createChrootIfNecessary) { - val chrootIndex = connectString.indexOf("/") - if (chrootIndex > 0) { - val zkConnWithoutChrootForChrootCreation = connectString.substring(0, chrootIndex) - val zkClientForChrootCreation = apply(zkConnWithoutChrootForChrootCreation, isSecure, sessionTimeoutMs, - connectionTimeoutMs, maxInFlightRequests, time, name, zkClientConfig, metricGroup, metricType) - try { - val chroot = connectString.substring(chrootIndex) - if (!zkClientForChrootCreation.pathExists(chroot)) { - zkClientForChrootCreation.makeSurePersistentPathExists(chroot) - } - } finally { - zkClientForChrootCreation.close() - } - } - } - val zooKeeperClient = new ZooKeeperClient(connectString, sessionTimeoutMs, connectionTimeoutMs, maxInFlightRequests, - time, metricGroup, metricType, zkClientConfig, name) - new KafkaZkClient(zooKeeperClient, isSecure, time, enableEntityConfigControllerCheck) - } - - // A helper function to transform a regular request into a MultiRequest - // with the check on controller epoch znode zkVersion. - // This is used for fencing zookeeper updates in controller. - private def wrapRequestWithControllerEpochCheck(request: AsyncRequest, expectedControllerZkVersion: Int): MultiRequest = { - val checkOp = CheckOp(ControllerEpochZNode.path, expectedControllerZkVersion) - request match { - case CreateRequest(path, data, acl, createMode, ctx) => - MultiRequest(Seq(checkOp, CreateOp(path, data, acl, createMode)), ctx) - case DeleteRequest(path, version, ctx) => - MultiRequest(Seq(checkOp, DeleteOp(path, version)), ctx) - case SetDataRequest(path, data, version, ctx) => - MultiRequest(Seq(checkOp, SetDataOp(path, data, version)), ctx) - case _ => throw new IllegalStateException(s"$request does not need controller epoch check") - } - } - - private def handleUnwrappedCheckOp(checkOp: CheckOp, checkOpResult: OpResult): Unit = { - checkOpResult match { - case errorResult: ErrorResult => - if (checkOp.path.equals(ControllerEpochZNode.path)) { - val errorCode = Code.get(errorResult.getErr) - if (errorCode == Code.BADVERSION) - // Throw ControllerMovedException when the zkVersionCheck is performed on the controller epoch znode and the check fails - throw new ControllerMovedException(s"Controller epoch zkVersion check fails. Expected zkVersion = ${checkOp.version}") - else if (errorCode != Code.OK) - throw KeeperException.create(errorCode, checkOp.path) - } - case _ => - } - } - - private def handleUnwrappedZkOp(zkOpResult: ZkOpResult, - resultCode: Code, - ctx: Option[Any], - responseMetadata: ResponseMetadata): AsyncResponse = { - val rawOpResult = zkOpResult.rawOpResult - zkOpResult.zkOp match { - case createOp: CreateOp => - val name = rawOpResult match { - case c: CreateResult => c.getPath - case _ => null - } - CreateResponse(resultCode, createOp.path, ctx, name, responseMetadata) - case deleteOp: DeleteOp => - DeleteResponse(resultCode, deleteOp.path, ctx, responseMetadata) - case setDataOp: SetDataOp => - val stat = rawOpResult match { - case s: SetDataResult => s.getStat - case _ => null - } - SetDataResponse(resultCode, setDataOp.path, ctx, stat, responseMetadata) - case zkOp => throw new IllegalStateException(s"Unexpected zkOp: $zkOp") - } - } - - // A helper function to transform a MultiResponse with the check on - // controller epoch znode zkVersion back into a regular response. - // ControllerMovedException will be thrown if the controller epoch - // znode zkVersion check fails. This is used for fencing zookeeper - // updates in controller. - private def unwrapResponseWithControllerEpochCheck(response: AsyncResponse): AsyncResponse = { - response match { - case MultiResponse(resultCode, _, ctx, zkOpResults, responseMetadata) => - zkOpResults match { - // In normal ZK writes, we just have a MultiOp with a CheckOp and the actual operation we're performing - case Seq(ZkOpResult(checkOp: CheckOp, checkOpResult), zkOpResult) => - handleUnwrappedCheckOp(checkOp, checkOpResult) - handleUnwrappedZkOp(zkOpResult, resultCode, ctx, responseMetadata) - case null => throw KeeperException.create(resultCode) - case _ => throw new IllegalStateException(s"Cannot unwrap $response because the first zookeeper op is not check op in original MultiRequest") - } - case _ => throw new IllegalStateException(s"Cannot unwrap $response because it is not a MultiResponse") - } - } - - def createZkClient(name: String, time: Time, config: KafkaConfig, zkClientConfig: ZKClientConfig): KafkaZkClient = { - val secureAclsEnabled = config.zkEnableSecureAcls - val isZkSecurityEnabled = JaasUtils.isZkSaslEnabled || KafkaConfig.zkTlsClientAuthEnabled(zkClientConfig) - - if (secureAclsEnabled && !isZkSecurityEnabled) - throw new java.lang.SecurityException( - s"${ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG} is true, but ZooKeeper client TLS configuration identifying at least " + - s"${ZkConfigs.ZK_SSL_CLIENT_ENABLE_CONFIG}, ${ZkConfigs.ZK_CLIENT_CNXN_SOCKET_CONFIG}, and " + - s"${ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG} was not present and the verification of the JAAS login file failed " + - s"${JaasUtils.zkSecuritySysConfigString}") - - KafkaZkClient(config.zkConnect, secureAclsEnabled, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, - config.zkMaxInFlightRequests, time, name = name, zkClientConfig = zkClientConfig, - createChrootIfNecessary = true) - } } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index d4c92150909..6633db51ad2 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -27,6 +27,7 @@ import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpo import kafka.server.DelegationTokenManagerZk import kafka.utils.Json import kafka.utils.json.JsonObject +import kafka.zookeeper.Stat import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.feature.Features._ import org.apache.kafka.common.feature.{Features, SupportedVersionRange} @@ -36,17 +37,13 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.token.delegation.TokenInformation import org.apache.kafka.common.utils.{SecurityUtils, Time} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState} +import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.common.{MetadataVersion, ProducerIdsBlock} -import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_2_7_IV0} import org.apache.kafka.server.config.ConfigType -import org.apache.zookeeper.ZooDefs -import org.apache.zookeeper.data.{ACL, Stat} import scala.beans.BeanProperty -import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, immutable, mutable} import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} @@ -113,17 +110,8 @@ object BrokerInfo { * any case). */ def apply(broker: Broker, metadataVersion: MetadataVersion, jmxPort: Int): BrokerInfo = { - val version = { - if (metadataVersion.isAtLeast(IBP_2_7_IV0)) - 5 - else if (metadataVersion.isAtLeast(IBP_0_10_0_IV1)) - 4 - else - 2 - } - BrokerInfo(broker, version, jmxPort) + throw new UnsupportedOperationException() } - } case class BrokerInfo(broker: Broker, version: Int, jmxPort: Int) { @@ -395,20 +383,7 @@ object TopicPartitionStateZNode { } def decode(bytes: Array[Byte], stat: Stat): Option[LeaderIsrAndControllerEpoch] = { - Json.parseBytes(bytes).map { js => - val leaderIsrAndEpochInfo = js.asJsonObject - val leader = leaderIsrAndEpochInfo("leader").to[Int] - val epoch = leaderIsrAndEpochInfo("leader_epoch").to[Int] - val isr = leaderIsrAndEpochInfo("isr").to[List[Int]] - val recovery = leaderIsrAndEpochInfo - .get("leader_recovery_state") - .map(jsonValue => LeaderRecoveryState.of(jsonValue.to[Int].toByte)) - .getOrElse(LeaderRecoveryState.RECOVERED) - val controllerEpoch = leaderIsrAndEpochInfo("controller_epoch").to[Int] - - val zkPathVersion = stat.getVersion - LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader, epoch, isr.map(Int.box).asJava, recovery, zkPathVersion), controllerEpoch) - } + throw new UnsupportedOperationException() } } @@ -763,8 +738,13 @@ case object ExtendedAclChangeStore extends ZkAclChangeStore { object ResourceZNode { def path(resource: ResourcePattern): String = ZkAclStore(resource.patternType).path(resource.resourceType, resource.name) - def encode(acls: Set[AclEntry]): Array[Byte] = Json.encodeAsBytes(AclEntry.toJsonCompatibleMap(acls.asJava)) - def decode(bytes: Array[Byte], stat: Stat): ZkData.VersionedAcls = ZkData.VersionedAcls(AclEntry.fromBytes(bytes).asScala.toSet, stat.getVersion) + def encode(acls: Set[AclEntry]): Array[Byte] = { + throw new UnsupportedOperationException() + } + + def decode(bytes: Array[Byte], stat: Stat): ZkData.VersionedAcls = { + throw new UnsupportedOperationException() + } } object ExtendedAclChangeEvent { @@ -1083,19 +1063,4 @@ object ZkData { ConfigEntityTypeZNode.path(ConfigType.BROKER), DelegationTokensZNode.path ) - - def sensitivePath(path: String): Boolean = { - path != null && SensitiveRootPaths.exists(path.startsWith) - } - - def defaultAcls(isSecure: Boolean, path: String): Seq[ACL] = { - //Old Consumer path is kept open as different consumers will write under this node. - if (!ConsumerPathZNode.path.equals(path) && isSecure) { - val acls = new ArrayBuffer[ACL] - acls ++= ZooDefs.Ids.CREATOR_ALL_ACL.asScala - if (!sensitivePath(path)) - acls ++= ZooDefs.Ids.READ_ACL_UNSAFE.asScala - acls - } else ZooDefs.Ids.OPEN_ACL_UNSAFE.asScala - } } diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala index 3f77acf3dbb..b6593f6c090 100755 --- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala +++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala @@ -17,30 +17,21 @@ package kafka.zookeeper -import java.util.Locale -import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock} -import java.util.concurrent._ -import java.util.{List => JList} -import kafka.utils.CoreUtils.{inLock, inReadLock, inWriteLock} import kafka.utils.Logging -import kafka.zookeeper.ZooKeeperClient._ import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.util.KafkaScheduler -import org.apache.kafka.server.metrics.KafkaMetricsGroup -import org.apache.zookeeper.AsyncCallback.{Children2Callback, DataCallback, StatCallback} -import org.apache.zookeeper.KeeperException.Code -import org.apache.zookeeper.Watcher.Event.{EventType, KeeperState} -import org.apache.zookeeper.ZooKeeper.States -import org.apache.zookeeper.data.{ACL, Stat} -import org.apache.zookeeper._ -import org.apache.zookeeper.client.ZKClientConfig -import scala.jdk.CollectionConverters._ -import scala.collection.{Seq, mutable} +import scala.collection.Seq -object ZooKeeperClient { - val RetryBackoffMs = 1000 +case class ACL() {} +case class CreateMode() {} +case class OpResult() {} +object Code { + val OK: Integer = 1 + val NONODE: Integer = 1 } +case class Code() {} +case class Stat() {} +case class KeeperException() extends RuntimeException {} /** * A ZooKeeper client that encourages pipelined requests. @@ -59,63 +50,9 @@ class ZooKeeperClient(connectString: String, time: Time, metricGroup: String, metricType: String, - private[zookeeper] val clientConfig: ZKClientConfig, name: String) extends Logging { - private val metricsGroup: KafkaMetricsGroup = new KafkaMetricsGroup(metricGroup, metricType) - this.logIdent = s"[ZooKeeperClient $name] " - private val initializationLock = new ReentrantReadWriteLock() - private val isConnectedOrExpiredLock = new ReentrantLock() - private val isConnectedOrExpiredCondition = isConnectedOrExpiredLock.newCondition() - private val zNodeChangeHandlers = new ConcurrentHashMap[String, ZNodeChangeHandler]().asScala - private val zNodeChildChangeHandlers = new ConcurrentHashMap[String, ZNodeChildChangeHandler]().asScala - private val inFlightRequests = new Semaphore(maxInFlightRequests) - private val stateChangeHandlers = new ConcurrentHashMap[String, StateChangeHandler]().asScala - private[zookeeper] val reinitializeScheduler = new KafkaScheduler(1, true, s"zk-client-${threadPrefix}reinit-") - private var isFirstConnectionEstablished = false - - private val metricNames = mutable.Set[String]() - - // The state map has to be created before creating ZooKeeper since it's needed in the ZooKeeper callback. - private val stateToMeterMap = { - import KeeperState._ - val stateToEventTypeMap = Map( - Disconnected -> "Disconnects", - SyncConnected -> "SyncConnects", - AuthFailed -> "AuthFailures", - ConnectedReadOnly -> "ReadOnlyConnects", - SaslAuthenticated -> "SaslAuthentications", - Expired -> "Expires" - ) - stateToEventTypeMap.map { case (state, eventType) => - val name = s"ZooKeeper${eventType}PerSec" - metricNames += name - state -> metricsGroup.newMeter(name, eventType.toLowerCase(Locale.ROOT), TimeUnit.SECONDS) - } - } - - info(s"Initializing a new session to $connectString.") - // Fail-fast if there's an error during construction (so don't call initialize, which retries forever) - @volatile private var zooKeeper = new ZooKeeper(connectString, sessionTimeoutMs, ZooKeeperClientWatcher, - clientConfig) - - metricsGroup.newGauge("SessionState", () => connectionState.toString) - - metricNames += "SessionState" - - reinitializeScheduler.startup() - try waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS) - catch { - case e: Throwable => - close() - throw e - } - - /** - * Return the state of the ZooKeeper connection. - */ - def connectionState: States = zooKeeper.getState /** * Send a request and wait for its response. See handle(Seq[AsyncRequest]) for details. @@ -124,7 +61,7 @@ class ZooKeeperClient(connectString: String, * @return an instance of the response with the specific type (e.g. CreateRequest -> CreateResponse). */ def handleRequest[Req <: AsyncRequest](request: Req): Req#Response = { - handleRequests(Seq(request)).head + throw new UnsupportedOperationException() } /** @@ -139,135 +76,7 @@ class ZooKeeperClient(connectString: String, * will be used (e.g. Seq[AsyncRequest] -> Seq[AsyncResponse]). */ def handleRequests[Req <: AsyncRequest](requests: Seq[Req]): Seq[Req#Response] = { - if (requests.isEmpty) - Seq.empty - else { - val countDownLatch = new CountDownLatch(requests.size) - val responseQueue = new ArrayBlockingQueue[Req#Response](requests.size) - - requests.foreach { request => - inFlightRequests.acquire() - try { - inReadLock(initializationLock) { - send(request) { response => - responseQueue.add(response) - inFlightRequests.release() - countDownLatch.countDown() - } - } - } catch { - case e: Throwable => - inFlightRequests.release() - throw e - } - } - countDownLatch.await() - responseQueue.asScala.toBuffer - } - } - - // Visibility to override for testing - private[zookeeper] def send[Req <: AsyncRequest](request: Req)(processResponse: Req#Response => Unit): Unit = { - // Safe to cast as we always create a response of the right type - def callback(response: AsyncResponse): Unit = processResponse(response.asInstanceOf[Req#Response]) - - def responseMetadata(sendTimeMs: Long) = ResponseMetadata(sendTimeMs, receivedTimeMs = time.hiResClockMs()) - - val sendTimeMs = time.hiResClockMs() - - // Cast to AsyncRequest to workaround a scalac bug that results in an false exhaustiveness warning - // with -Xlint:strict-unsealed-patmat - (request: AsyncRequest) match { - case ExistsRequest(path, ctx) => - zooKeeper.exists(path, shouldWatch(request), new StatCallback { - def processResult(rc: Int, path: String, ctx: Any, stat: Stat): Unit = - callback(ExistsResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs))) - }, ctx.orNull) - case GetDataRequest(path, ctx) => - zooKeeper.getData(path, shouldWatch(request), new DataCallback { - def processResult(rc: Int, path: String, ctx: Any, data: Array[Byte], stat: Stat): Unit = - callback(GetDataResponse(Code.get(rc), path, Option(ctx), data, stat, responseMetadata(sendTimeMs))) - }, ctx.orNull) - case GetChildrenRequest(path, _, ctx) => - zooKeeper.getChildren(path, shouldWatch(request), new Children2Callback { - def processResult(rc: Int, path: String, ctx: Any, children: JList[String], stat: Stat): Unit = - callback(GetChildrenResponse(Code.get(rc), path, Option(ctx), Option(children).map(_.asScala).getOrElse(Seq.empty), - stat, responseMetadata(sendTimeMs))) - }, ctx.orNull) - case CreateRequest(path, data, acl, createMode, ctx) => - zooKeeper.create(path, data, acl.asJava, createMode, - (rc, path, ctx, name) => - callback(CreateResponse(Code.get(rc), path, Option(ctx), name, responseMetadata(sendTimeMs))), - ctx.orNull) - case SetDataRequest(path, data, version, ctx) => - zooKeeper.setData(path, data, version, - (rc, path, ctx, stat) => - callback(SetDataResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs))), - ctx.orNull) - case DeleteRequest(path, version, ctx) => - zooKeeper.delete(path, version, - (rc, path, ctx) => callback(DeleteResponse(Code.get(rc), path, Option(ctx), responseMetadata(sendTimeMs))), - ctx.orNull) - case GetAclRequest(path, ctx) => - zooKeeper.getACL(path, null, - (rc, path, ctx, acl, stat) => - callback(GetAclResponse(Code.get(rc), path, Option(ctx), Option(acl).map(_.asScala).getOrElse(Seq.empty), - stat, responseMetadata(sendTimeMs))), - ctx.orNull) - case SetAclRequest(path, acl, version, ctx) => - zooKeeper.setACL(path, acl.asJava, version, - (rc, path, ctx, stat) => - callback(SetAclResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs))), - ctx.orNull) - case MultiRequest(zkOps, ctx) => - def toZkOpResult(opResults: JList[OpResult]): Seq[ZkOpResult] = - Option(opResults).map(results => zkOps.zip(results.asScala).map { case (zkOp, result) => - ZkOpResult(zkOp, result) - }).orNull - zooKeeper.multi(zkOps.map(_.toZookeeperOp).asJava, - (rc, path, ctx, opResults) => - callback(MultiResponse(Code.get(rc), path, Option(ctx), toZkOpResult(opResults), responseMetadata(sendTimeMs))), - ctx.orNull) - } - } - - /** - * Wait indefinitely until the underlying zookeeper client to reaches the CONNECTED state. - * @throws ZooKeeperClientAuthFailedException if the authentication failed either before or while waiting for connection. - * @throws ZooKeeperClientExpiredException if the session expired either before or while waiting for connection. - */ - def waitUntilConnected(): Unit = inLock(isConnectedOrExpiredLock) { - waitUntilConnected(Long.MaxValue, TimeUnit.MILLISECONDS) - } - - private def waitUntilConnected(timeout: Long, timeUnit: TimeUnit): Unit = { - info("Waiting until connected.") - var nanos = timeUnit.toNanos(timeout) - inLock(isConnectedOrExpiredLock) { - var state = connectionState - while (!state.isConnected && state.isAlive) { - if (nanos <= 0) { - throw new ZooKeeperClientTimeoutException(s"Timed out waiting for connection while in state: $state") - } - nanos = isConnectedOrExpiredCondition.awaitNanos(nanos) - state = connectionState - } - if (state == States.AUTH_FAILED) { - throw new ZooKeeperClientAuthFailedException("Auth failed either before or while waiting for connection") - } else if (state == States.CLOSED) { - throw new ZooKeeperClientExpiredException("Session expired either before or while waiting for connection") - } - isFirstConnectionEstablished = true - } - info("Connected.") - } - - // If this method is changed, the documentation for registerZNodeChangeHandler and/or registerZNodeChildChangeHandler - // may need to be updated. - private def shouldWatch(request: AsyncRequest): Boolean = request match { - case GetChildrenRequest(_, registerWatch, _) => registerWatch && zNodeChildChangeHandlers.contains(request.path) - case _: ExistsRequest | _: GetDataRequest => zNodeChangeHandlers.contains(request.path) - case _ => throw new IllegalArgumentException(s"Request $request is not watchable") + throw new UnsupportedOperationException() } /** @@ -281,7 +90,7 @@ class ZooKeeperClient(connectString: String, * @param zNodeChangeHandler the handler to register */ def registerZNodeChangeHandler(zNodeChangeHandler: ZNodeChangeHandler): Unit = { - zNodeChangeHandlers.put(zNodeChangeHandler.path, zNodeChangeHandler) + throw new UnsupportedOperationException() } /** @@ -289,7 +98,7 @@ class ZooKeeperClient(connectString: String, * @param path the path of the handler to unregister */ def unregisterZNodeChangeHandler(path: String): Unit = { - zNodeChangeHandlers.remove(path) + throw new UnsupportedOperationException() } /** @@ -300,7 +109,7 @@ class ZooKeeperClient(connectString: String, * @param zNodeChildChangeHandler the handler to register */ def registerZNodeChildChangeHandler(zNodeChildChangeHandler: ZNodeChildChangeHandler): Unit = { - zNodeChildChangeHandlers.put(zNodeChildChangeHandler.path, zNodeChildChangeHandler) + throw new UnsupportedOperationException() } /** @@ -308,148 +117,30 @@ class ZooKeeperClient(connectString: String, * @param path the path of the handler to unregister */ def unregisterZNodeChildChangeHandler(path: String): Unit = { - zNodeChildChangeHandlers.remove(path) + throw new UnsupportedOperationException() } /** * @param stateChangeHandler */ - def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = inReadLock(initializationLock) { - if (stateChangeHandler != null) - stateChangeHandlers.put(stateChangeHandler.name, stateChangeHandler) + def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { + throw new UnsupportedOperationException() } /** * * @param name */ - def unregisterStateChangeHandler(name: String): Unit = inReadLock(initializationLock) { - stateChangeHandlers.remove(name) + def unregisterStateChangeHandler(name: String): Unit = { + throw new UnsupportedOperationException() } def close(): Unit = { - info("Closing.") - - // Shutdown scheduler outside of lock to avoid deadlock if scheduler - // is waiting for lock to process session expiry. Close expiry thread - // first to ensure that new clients are not created during close(). - reinitializeScheduler.shutdown() - - inWriteLock(initializationLock) { - zNodeChangeHandlers.clear() - zNodeChildChangeHandlers.clear() - stateChangeHandlers.clear() - zooKeeper.close() - metricNames.foreach(metricsGroup.removeMetric) - } - info("Closed.") + throw new UnsupportedOperationException() } - def sessionId: Long = inReadLock(initializationLock) { - zooKeeper.getSessionId - } - - // Only for testing - private[kafka] def currentZooKeeper: ZooKeeper = inReadLock(initializationLock) { - zooKeeper - } - - private def reinitialize(): Unit = { - // Initialization callbacks are invoked outside of the lock to avoid deadlock potential since their completion - // may require additional Zookeeper requests, which will block to acquire the initialization lock - stateChangeHandlers.values.foreach(callBeforeInitializingSession) - - inWriteLock(initializationLock) { - if (!connectionState.isAlive) { - zooKeeper.close() - info(s"Initializing a new session to $connectString.") - // retry forever until ZooKeeper can be instantiated - var connected = false - while (!connected) { - try { - zooKeeper = new ZooKeeper(connectString, sessionTimeoutMs, ZooKeeperClientWatcher, clientConfig) - connected = true - } catch { - case e: Exception => - info("Error when recreating ZooKeeper, retrying after a short sleep", e) - Thread.sleep(RetryBackoffMs) - } - } - } - } - - stateChangeHandlers.values.foreach(callAfterInitializingSession) - } - - /** - * Close the zookeeper client to force session reinitialization. This is visible for testing only. - */ - private[zookeeper] def forceReinitialize(): Unit = { - zooKeeper.close() - reinitialize() - } - - private def callBeforeInitializingSession(handler: StateChangeHandler): Unit = { - try { - handler.beforeInitializingSession() - } catch { - case t: Throwable => - error(s"Uncaught error in handler ${handler.name}", t) - } - } - - private def callAfterInitializingSession(handler: StateChangeHandler): Unit = { - try { - handler.afterInitializingSession() - } catch { - case t: Throwable => - error(s"Uncaught error in handler ${handler.name}", t) - } - } - - // Visibility for testing - private[zookeeper] def scheduleReinitialize(name: String, message: String, delayMs: Long): Unit = { - reinitializeScheduler.scheduleOnce(name, () => { - info(message) - reinitialize() - }, delayMs) - } - - private def threadPrefix: String = name.replaceAll("\\s", "") + "-" - - // package level visibility for testing only - private[zookeeper] object ZooKeeperClientWatcher extends Watcher { - override def process(event: WatchedEvent): Unit = { - debug(s"Received event: $event") - Option(event.getPath) match { - case None => - val state = event.getState - stateToMeterMap.get(state).foreach(_.mark()) - inLock(isConnectedOrExpiredLock) { - isConnectedOrExpiredCondition.signalAll() - } - if (state == KeeperState.AuthFailed) { - error(s"Auth failed, initialized=$isFirstConnectionEstablished connectionState=$connectionState") - stateChangeHandlers.values.foreach(_.onAuthFailure()) - - // If this is during initial startup, we fail fast. Otherwise, schedule retry. - val initialized = inLock(isConnectedOrExpiredLock) { - isFirstConnectionEstablished - } - if (initialized && !connectionState.isAlive) - scheduleReinitialize("auth-failed", "Reinitializing due to auth failure.", RetryBackoffMs) - } else if (state == KeeperState.Expired) { - scheduleReinitialize("session-expired", "Session expired.", delayMs = 0L) - } - case Some(path) => - (event.getType: @unchecked) match { - case EventType.NodeChildrenChanged => zNodeChildChangeHandlers.get(path).foreach(_.handleChildChange()) - case EventType.NodeCreated => zNodeChangeHandlers.get(path).foreach(_.handleCreation()) - case EventType.NodeDeleted => zNodeChangeHandlers.get(path).foreach(_.handleDeletion()) - case EventType.NodeDataChanged => zNodeChangeHandlers.get(path).foreach(_.handleDataChange()) - } - } - } + def sessionId: Long = { + throw new UnsupportedOperationException() } } @@ -474,23 +165,18 @@ trait ZNodeChildChangeHandler { // Thin wrapper for zookeeper.Op sealed trait ZkOp { - def toZookeeperOp: Op } case class CreateOp(path: String, data: Array[Byte], acl: Seq[ACL], createMode: CreateMode) extends ZkOp { - override def toZookeeperOp: Op = Op.create(path, data, acl.asJava, createMode) } case class DeleteOp(path: String, version: Int) extends ZkOp { - override def toZookeeperOp: Op = Op.delete(path, version) } case class SetDataOp(path: String, data: Array[Byte], version: Int) extends ZkOp { - override def toZookeeperOp: Op = Op.setData(path, data, version) } case class CheckOp(path: String, version: Int) extends ZkOp { - override def toZookeeperOp: Op = Op.check(path, version) } case class ZkOpResult(zkOp: ZkOp, rawOpResult: OpResult) @@ -550,19 +236,9 @@ sealed abstract class AsyncResponse { def path: String def ctx: Option[Any] - /** Return None if the result code is OK and KeeperException otherwise. */ - def resultException: Option[KeeperException] = - if (resultCode == Code.OK) None else Some(KeeperException.create(resultCode, path)) - - /** - * Throw KeeperException if the result code is not OK. - */ - def maybeThrow(): Unit = { - if (resultCode != Code.OK) - throw KeeperException.create(resultCode, path) - } - def metadata: ResponseMetadata + + def resultException: Option[RuntimeException] = None } case class ResponseMetadata(sendTimeMs: Long, receivedTimeMs: Long) { @@ -588,7 +264,4 @@ case class GetChildrenResponse(resultCode: Code, path: String, ctx: Option[Any], case class MultiResponse(resultCode: Code, path: String, ctx: Option[Any], zkOpResults: Seq[ZkOpResult], metadata: ResponseMetadata) extends AsyncResponse -class ZooKeeperClientException(message: String) extends RuntimeException(message) -class ZooKeeperClientExpiredException(message: String) extends ZooKeeperClientException(message) -class ZooKeeperClientAuthFailedException(message: String) extends ZooKeeperClientException(message) -class ZooKeeperClientTimeoutException(message: String) extends ZooKeeperClientException(message) +case class ZooKeeperClientException(message: String) extends RuntimeException(message) \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index a38a9189ce6..84eb0ae4aaa 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -20,7 +20,6 @@ package kafka.api import kafka.security.JaasTestUtils import kafka.security.JaasTestUtils.JaasSection import kafka.security.minikdc.MiniKdc -import kafka.server.KafkaConfig import kafka.utils.TestUtils import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, ScramCredentialInfo, UserScramCredentialUpsertion, ScramMechanism => PublicScramMechanism} @@ -30,9 +29,7 @@ import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.authenticator.LoginManager import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism} -import org.apache.kafka.common.utils.Time import org.apache.kafka.server.config.ConfigType -import org.apache.zookeeper.client.ZKClientConfig import java.io.File import java.util @@ -203,10 +200,7 @@ trait SaslSetup { } def createScramCredentials(zkConnect: String, userName: String, password: String): Unit = { - val zkClientConfig = new ZKClientConfig() - Using.resource(KafkaZkClient( - zkConnect, JaasUtils.isZkSaslEnabled || KafkaConfig.zkTlsClientAuthEnabled(zkClientConfig), 30000, 30000, - Int.MaxValue, Time.SYSTEM, name = "SaslSetup", zkClientConfig = zkClientConfig, enableEntityConfigControllerCheck = false)) { zkClient => + Using.resource(new KafkaZkClient()) { zkClient => val adminZkClient = new AdminZkClient(zkClient) val entityType = ConfigType.USER diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 85238c3439e..b2db72d0685 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -22,7 +22,6 @@ import kafka.network.SocketServer import kafka.server.IntegrationTestUtils.connectAndReceive import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils -import org.apache.commons.io.FileUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter} @@ -57,7 +56,7 @@ import org.slf4j.LoggerFactory import java.io.File import java.nio.charset.StandardCharsets -import java.nio.file.{FileSystems, Files, Path} +import java.nio.file.{FileSystems, Files, Path, Paths} import java.{lang, util} import java.util.concurrent.{CompletableFuture, CompletionStage, ExecutionException, TimeUnit} import java.util.concurrent.atomic.AtomicInteger @@ -1435,6 +1434,15 @@ class KRaftClusterTest { } } + def copyDirectory(src: String, dest: String): Unit = { + Files.walk(Paths.get(src)).forEach(p => { + val out = Paths.get(dest, p.toString().substring(src.length())) + if (!p.toString().equals(src)) { + Files.copy(p, out); + } + }); + } + @Test def testAbandonedFutureReplicaRecovered_mainReplicaInOnlineLogDir(): Unit = { val cluster = new KafkaClusterTestKit.Builder( @@ -1476,7 +1484,8 @@ class KRaftClusterTest { val parentDir = log.parentDir val targetParentDir = broker0.config.logDirs.filter(_ != parentDir).head val targetDirFile = new File(targetParentDir, log.dir.getName) - FileUtils.copyDirectory(log.dir, targetDirFile) + targetDirFile.mkdir() + copyDirectory(log.dir.toString(), targetDirFile.toString()) assertTrue(targetDirFile.exists()) // Rename original log to a future diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 52ef2fcfde1..b541de098ed 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -60,8 +60,6 @@ versions += [ caffeine: "3.1.1", bndlib: "7.0.0", checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2", - commonsCli: "1.4", - commonsIo: "2.14.0", // ZooKeeper dependency. Do not use, this is going away. commonsValidator: "1.9.0", classgraph: "4.8.173", dropwizardMetrics: "4.1.12.1", @@ -114,7 +112,6 @@ versions += [ mavenArtifact: "3.9.6", metrics: "2.2.0", mockito: "5.14.2", - netty: "4.1.115.Final", opentelemetryProto: "1.0.0-alpha", protobuf: "3.25.5", // a dependency of opentelemetryProto pcollections: "4.0.1", @@ -129,7 +126,6 @@ versions += [ snappy: "1.1.10.5", spotbugs: "4.8.6", zinc: "1.9.2", - zookeeper: "3.8.4", // When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json // Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid zstd: "1.5.6-6", @@ -153,8 +149,6 @@ libs += [ bndlib:"biz.aQute.bnd:biz.aQute.bndlib:$versions.bndlib", caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine", classgraph: "io.github.classgraph:classgraph:$versions.classgraph", - commonsCli: "commons-cli:commons-cli:$versions.commonsCli", - commonsIo: "commons-io:commons-io:$versions.commonsIo", commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", @@ -218,8 +212,6 @@ libs += [ dropwizardMetrics: "io.dropwizard.metrics:metrics-core:$versions.dropwizardMetrics", mockitoCore: "org.mockito:mockito-core:$versions.mockito", mockitoJunitJupiter: "org.mockito:mockito-junit-jupiter:$versions.mockito", - nettyHandler: "io.netty:netty-handler:$versions.netty", - nettyTransportNativeEpoll: "io.netty:netty-transport-native-epoll:$versions.netty", pcollections: "org.pcollections:pcollections:$versions.pcollections", opentelemetryProto: "io.opentelemetry.proto:opentelemetry-proto:$versions.opentelemetryProto", protobuf: "com.google.protobuf:protobuf-java:$versions.protobuf", @@ -233,7 +225,6 @@ libs += [ snappy: "org.xerial.snappy:snappy-java:$versions.snappy", spotbugs: "com.github.spotbugs:spotbugs-annotations:$versions.spotbugs", swaggerAnnotations: "io.swagger.core.v3:swagger-annotations:$swaggerVersion", - zookeeper: "org.apache.zookeeper:zookeeper:$versions.zookeeper", jfreechart: "jfreechart:jfreechart:$versions.jfreechart", mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact", zstd: "com.github.luben:zstd-jni:$versions.zstd",