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",