From 0e502e0b477a5af5e7c545ddab03de41ad2be481 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Fri, 17 Jan 2025 10:56:05 +0800 Subject: [PATCH] KAFKA-18431: Remove KafkaController (#18573) Remove KafkaController and related unused references: * ControllerChannelContext * ControllerChannelManager * ControllerEventManager * ControllerState * PartitionStateMachine * ReplicaStateMachine * TopicDeletionManager * ZkBrokerEpochManager Reviewers: Ismael Juma --- .../controller/ControllerChannelContext.scala | 42 - .../controller/ControllerChannelManager.scala | 758 ----- .../kafka/controller/ControllerContext.scala | 497 +-- .../controller/ControllerEventManager.scala | 168 - .../kafka/controller/ControllerState.scala | 122 - .../kafka/controller/KafkaController.scala | 2857 ----------------- .../controller/PartitionStateMachine.scala | 156 - .../controller/ReplicaStateMachine.scala | 115 - .../controller/TopicDeletionManager.scala | 357 -- .../src/main/scala/kafka/log/LogManager.scala | 7 +- .../scala/kafka/server/BrokerServer.scala | 2 +- .../scala/kafka/server/ConfigHandler.scala | 46 +- .../scala/kafka/server/ReplicaManager.scala | 47 +- .../main/scala/kafka/zk/KafkaZkClient.scala | 646 +--- core/src/main/scala/kafka/zk/ZkData.scala | 1003 ------ .../kafka/server/QuorumTestHarness.scala | 4 +- .../kafka/cluster/BrokerEndPointTest.scala | 204 -- .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- .../server/DynamicConfigChangeTest.scala | 12 +- 19 files changed, 18 insertions(+), 7027 deletions(-) delete mode 100644 core/src/main/scala/kafka/controller/ControllerChannelContext.scala delete mode 100755 core/src/main/scala/kafka/controller/ControllerChannelManager.scala delete mode 100644 core/src/main/scala/kafka/controller/ControllerEventManager.scala delete mode 100644 core/src/main/scala/kafka/controller/ControllerState.scala delete mode 100644 core/src/main/scala/kafka/controller/KafkaController.scala delete mode 100755 core/src/main/scala/kafka/controller/PartitionStateMachine.scala delete mode 100644 core/src/main/scala/kafka/controller/ReplicaStateMachine.scala delete mode 100755 core/src/main/scala/kafka/controller/TopicDeletionManager.scala delete mode 100644 core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala diff --git a/core/src/main/scala/kafka/controller/ControllerChannelContext.scala b/core/src/main/scala/kafka/controller/ControllerChannelContext.scala deleted file mode 100644 index 0ab43256290a4..0000000000000 --- a/core/src/main/scala/kafka/controller/ControllerChannelContext.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.controller - -import kafka.cluster.Broker -import org.apache.kafka.common.{TopicPartition, Uuid} - -trait ControllerChannelContext { - def isTopicDeletionInProgress(topicName: String): Boolean - - def topicIds: collection.Map[String, Uuid] - - def liveBrokerIdAndEpochs: collection.Map[Int, Long] - - def liveOrShuttingDownBrokers: collection.Set[Broker] - - def isTopicQueuedUpForDeletion(topic: String): Boolean - - def isReplicaOnline(brokerId: Int, partition: TopicPartition): Boolean - - def partitionReplicaAssignment(partition: TopicPartition): collection.Seq[Int] - - def leaderEpoch(topicPartition: TopicPartition): Int - - def liveOrShuttingDownBrokerIds: collection.Set[Int] - - def partitionLeadershipInfo(topicPartition: TopicPartition): Option[LeaderIsrAndControllerEpoch] -} diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala deleted file mode 100755 index 8417d513348ec..0000000000000 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ /dev/null @@ -1,758 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.controller - -import com.yammer.metrics.core.{Gauge, Timer} -import kafka.cluster.Broker -import kafka.server.KafkaConfig -import kafka.utils._ -import org.apache.kafka.clients._ -import org.apache.kafka.common._ -import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState -import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} -import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} -import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.network._ -import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests._ -import org.apache.kafka.common.security.JaasContext -import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.metadata.LeaderAndIsr -import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.common.MetadataVersion._ -import org.apache.kafka.server.metrics.KafkaMetricsGroup -import org.apache.kafka.server.util.ShutdownableThread - -import java.net.SocketTimeoutException -import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue, TimeUnit} -import scala.collection.{Seq, Set, mutable} -import scala.jdk.CollectionConverters._ - -object ControllerChannelManager { - private val QueueSizeMetricName = "QueueSize" - private val RequestRateAndQueueTimeMetricName = "RequestRateAndQueueTimeMs" -} - -class ControllerChannelManager(controllerEpoch: () => Int, - config: KafkaConfig, - time: Time, - metrics: Metrics, - stateChangeLogger: StateChangeLogger, - threadNamePrefix: Option[String] = None) extends Logging { - import ControllerChannelManager._ - - private val metricsGroup = new KafkaMetricsGroup(this.getClass) - - protected val brokerStateInfo = new mutable.HashMap[Int, ControllerBrokerStateInfo] - private val brokerLock = new Object - this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " - - metricsGroup.newGauge("TotalQueueSize", - () => brokerLock synchronized { - brokerStateInfo.values.iterator.map(_.messageQueue.size).sum - } - ) - - def startup(initialBrokers: Set[Broker]):Unit = { - initialBrokers.foreach(addNewBroker) - - brokerLock synchronized { - brokerStateInfo.foreach(brokerState => startRequestSendThread(brokerState._1)) - } - } - - def shutdown():Unit = { - brokerLock synchronized { - brokerStateInfo.values.toList.foreach(removeExistingBroker) - } - } - - def sendRequest(brokerId: Int, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], - callback: AbstractResponse => Unit = null): Unit = { - brokerLock synchronized { - val stateInfoOpt = brokerStateInfo.get(brokerId) - stateInfoOpt match { - case Some(stateInfo) => - stateInfo.messageQueue.put(QueueItem(request.apiKey, request, callback, time.milliseconds())) - case None => - warn(s"Not sending request ${request.apiKey.name} with controllerId=${request.controllerId()}, " + - s"controllerEpoch=${request.controllerEpoch()}, brokerEpoch=${request.brokerEpoch()} " + - s"to broker $brokerId, since it is offline.") - } - } - } - - def addBroker(broker: Broker): Unit = { - // be careful here. Maybe the startup() API has already started the request send thread - brokerLock synchronized { - if (!brokerStateInfo.contains(broker.id)) { - addNewBroker(broker) - startRequestSendThread(broker.id) - } - } - } - - def removeBroker(brokerId: Int): Unit = { - brokerLock synchronized { - removeExistingBroker(brokerStateInfo(brokerId)) - } - } - - private def addNewBroker(broker: Broker): Unit = { - val messageQueue = new LinkedBlockingQueue[QueueItem] - debug(s"Controller ${config.brokerId} trying to connect to broker ${broker.id}") - val controllerToBrokerListenerName = config.interBrokerListenerName - val controllerToBrokerSecurityProtocol = config.interBrokerSecurityProtocol - val brokerNode = broker.node(controllerToBrokerListenerName) - val logContext = new LogContext(s"[Controller id=${config.brokerId}, targetBrokerId=${brokerNode.idString}] ") - val (networkClient, reconfigurableChannelBuilder) = { - val channelBuilder = ChannelBuilders.clientChannelBuilder( - controllerToBrokerSecurityProtocol, - JaasContext.Type.SERVER, - config, - controllerToBrokerListenerName, - config.saslMechanismInterBrokerProtocol, - time, - logContext - ) - val reconfigurableChannelBuilder = channelBuilder match { - case reconfigurable: Reconfigurable => - config.addReconfigurable(reconfigurable) - Some(reconfigurable) - case _ => None - } - val selector = new Selector( - NetworkReceive.UNLIMITED, - Selector.NO_IDLE_TIMEOUT_MS, - metrics, - time, - "controller-channel", - Map("broker-id" -> brokerNode.idString).asJava, - false, - channelBuilder, - logContext - ) - val networkClient = new NetworkClient( - selector, - new ManualMetadataUpdater(Seq(brokerNode).asJava), - config.brokerId.toString, - 1, - 0, - 0, - Selectable.USE_DEFAULT_BUFFER_SIZE, - Selectable.USE_DEFAULT_BUFFER_SIZE, - config.requestTimeoutMs, - config.connectionSetupTimeoutMs, - config.connectionSetupTimeoutMaxMs, - time, - false, - new ApiVersions, - logContext, - MetadataRecoveryStrategy.NONE - ) - (networkClient, reconfigurableChannelBuilder) - } - val threadName = threadNamePrefix match { - case None => s"Controller-${config.brokerId}-to-broker-${broker.id}-send-thread" - case Some(name) => s"$name:Controller-${config.brokerId}-to-broker-${broker.id}-send-thread" - } - - val requestRateAndQueueTimeMetrics = metricsGroup.newTimer( - RequestRateAndQueueTimeMetricName, TimeUnit.MILLISECONDS, TimeUnit.SECONDS, brokerMetricTags(broker.id) - ) - - val requestThread = new RequestSendThread(config.brokerId, controllerEpoch, messageQueue, networkClient, - brokerNode, config, time, requestRateAndQueueTimeMetrics, stateChangeLogger, threadName) - requestThread.setDaemon(false) - - val queueSizeGauge = metricsGroup.newGauge(QueueSizeMetricName, () => messageQueue.size, brokerMetricTags(broker.id)) - - brokerStateInfo.put(broker.id, ControllerBrokerStateInfo(networkClient, brokerNode, messageQueue, - requestThread, queueSizeGauge, requestRateAndQueueTimeMetrics, reconfigurableChannelBuilder)) - } - - private def brokerMetricTags(brokerId: Int) = Map("broker-id" -> brokerId.toString).asJava - - private def removeExistingBroker(brokerState: ControllerBrokerStateInfo): Unit = { - try { - // Shutdown the RequestSendThread before closing the NetworkClient to avoid the concurrent use of the - // non-threadsafe classes as described in KAFKA-4959. - // The call to shutdownLatch.await() in ShutdownableThread.shutdown() serves as a synchronization barrier that - // hands off the NetworkClient from the RequestSendThread to the ZkEventThread. - brokerState.reconfigurableChannelBuilder.foreach(config.removeReconfigurable) - brokerState.requestSendThread.shutdown() - brokerState.networkClient.close() - brokerState.messageQueue.clear() - metricsGroup.removeMetric(QueueSizeMetricName, brokerMetricTags(brokerState.brokerNode.id)) - metricsGroup.removeMetric(RequestRateAndQueueTimeMetricName, brokerMetricTags(brokerState.brokerNode.id)) - brokerStateInfo.remove(brokerState.brokerNode.id) - } catch { - case e: Throwable => error("Error while removing broker by the controller", e) - } - } - - private def startRequestSendThread(brokerId: Int): Unit = { - val requestThread = brokerStateInfo(brokerId).requestSendThread - if (requestThread.getState == Thread.State.NEW) - requestThread.start() - } -} - -case class QueueItem(apiKey: ApiKeys, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], - callback: AbstractResponse => Unit, enqueueTimeMs: Long) - -class RequestSendThread(val controllerId: Int, - controllerEpoch: () => Int, - val queue: BlockingQueue[QueueItem], - val networkClient: NetworkClient, - val brokerNode: Node, - val config: KafkaConfig, - val time: Time, - val requestRateAndQueueTimeMetrics: Timer, - val stateChangeLogger: StateChangeLogger, - name: String) - extends ShutdownableThread(name, true, s"[RequestSendThread controllerId=$controllerId] ") - with Logging { - - logIdent = logPrefix - - private val socketTimeoutMs = config.controllerSocketTimeoutMs - - override def doWork(): Unit = { - - def backoff(): Unit = pause(100, TimeUnit.MILLISECONDS) - - val QueueItem(apiKey, requestBuilder, callback, enqueueTimeMs) = queue.take() - requestRateAndQueueTimeMetrics.update(time.milliseconds() - enqueueTimeMs, TimeUnit.MILLISECONDS) - - var clientResponse: ClientResponse = null - try { - var isSendSuccessful = false - while (isRunning && !isSendSuccessful) { - // if a broker goes down for a long time, then at some point the controller's zookeeper listener will trigger a - // removeBroker which will invoke shutdown() on this thread. At that point, we will stop retrying. - try { - if (!brokerReady()) { - isSendSuccessful = false - backoff() - } - else { - val clientRequest = networkClient.newClientRequest(brokerNode.idString, requestBuilder, - time.milliseconds(), true) - clientResponse = NetworkClientUtils.sendAndReceive(networkClient, clientRequest, time) - isSendSuccessful = true - } - } catch { - case e: Throwable => // if the send was not successful, reconnect to broker and resend the message - warn(s"Controller $controllerId epoch ${controllerEpoch()} fails to send request " + - s"$requestBuilder " + - s"to broker $brokerNode. Reconnecting to broker.", e) - networkClient.close(brokerNode.idString) - isSendSuccessful = false - backoff() - } - } - if (clientResponse != null) { - val requestHeader = clientResponse.requestHeader - val api = requestHeader.apiKey - if (api != ApiKeys.LEADER_AND_ISR && api != ApiKeys.STOP_REPLICA && api != ApiKeys.UPDATE_METADATA) - throw new KafkaException(s"Unexpected apiKey received: $apiKey") - - val response = clientResponse.responseBody - - stateChangeLogger.withControllerEpoch(controllerEpoch()).trace(s"Received response " + - s"$response for request $api with correlation id " + - s"${requestHeader.correlationId} sent to broker $brokerNode") - - if (callback != null) { - callback(response) - } - } - } catch { - case e: Throwable => - error(s"Controller $controllerId fails to send a request to broker $brokerNode", e) - // If there is any socket error (eg, socket timeout), the connection is no longer usable and needs to be recreated. - networkClient.close(brokerNode.idString) - } - } - - private def brokerReady(): Boolean = { - try { - if (!NetworkClientUtils.isReady(networkClient, brokerNode, time.milliseconds())) { - if (!NetworkClientUtils.awaitReady(networkClient, brokerNode, time, socketTimeoutMs)) - throw new SocketTimeoutException(s"Failed to connect within $socketTimeoutMs ms") - - info(s"Controller $controllerId connected to $brokerNode for sending state change requests") - } - - true - } catch { - case e: Throwable => - warn(s"Controller $controllerId's connection to broker $brokerNode was unsuccessful", e) - networkClient.close(brokerNode.idString) - false - } - } - - override def initiateShutdown(): Boolean = { - if (super.initiateShutdown()) { - networkClient.initiateClose() - true - } else - false - } -} - -class ControllerBrokerRequestBatch( - config: KafkaConfig, - controllerChannelManager: ControllerChannelManager, - controllerEventManager: ControllerEventManager, - controllerContext: ControllerContext, - stateChangeLogger: StateChangeLogger -) extends AbstractControllerBrokerRequestBatch( - config, - () => controllerContext, - () => config.interBrokerProtocolVersion, - stateChangeLogger -) { - - private def sendEvent(event: ControllerEvent): Unit = { - controllerEventManager.put(event) - } - def sendRequest(brokerId: Int, - request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], - callback: AbstractResponse => Unit = null): Unit = { - controllerChannelManager.sendRequest(brokerId, request, callback) - } - - override def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit = { - sendEvent(LeaderAndIsrResponseReceived(response, broker)) - } - - override def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int): Unit = { - sendEvent(UpdateMetadataResponseReceived(response, broker)) - } - - override def handleStopReplicaResponse(stopReplicaResponse: StopReplicaResponse, brokerId: Int, - partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit = { - if (partitionErrorsForDeletingTopics.nonEmpty) - sendEvent(TopicDeletionStopReplicaResponseReceived(brokerId, stopReplicaResponse.error, partitionErrorsForDeletingTopics)) - } -} - -/** - * Structure to send RPCs from controller to broker to inform about the metadata and leadership - * changes in the system. - * @param config Kafka config present in the controller. - * @param metadataProvider Provider to provide the relevant metadata to build the state needed to - * send RPCs - * @param metadataVersionProvider Provider to provide the metadata version used by the controller. - * @param stateChangeLogger logger to log the various events while sending requests and receiving - * responses from the brokers - * @param kraftController whether the controller is KRaft controller - */ -abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, - metadataProvider: () => ControllerChannelContext, - metadataVersionProvider: () => MetadataVersion, - stateChangeLogger: StateChangeLogger, - kraftController: Boolean = false) extends Logging { - val controllerId: Int = config.brokerId - private val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]] - private val stopReplicaRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, StopReplicaPartitionState]] - private val updateMetadataRequestBrokerSet = mutable.Set.empty[Int] - private val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, UpdateMetadataPartitionState] - private var updateType: AbstractControlRequest.Type = AbstractControlRequest.Type.UNKNOWN - private var metadataInstance: ControllerChannelContext = _ - - def sendRequest(brokerId: Int, - request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], - callback: AbstractResponse => Unit = null): Unit - - def newBatch(): Unit = { - // raise error if the previous batch is not empty - if (leaderAndIsrRequestMap.nonEmpty) - throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating " + - s"a new one. Some LeaderAndIsr state changes $leaderAndIsrRequestMap might be lost ") - if (stopReplicaRequestMap.nonEmpty) - throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + - s"new one. Some StopReplica state changes $stopReplicaRequestMap might be lost ") - if (updateMetadataRequestBrokerSet.nonEmpty) - throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + - s"new one. Some UpdateMetadata state changes to brokers $updateMetadataRequestBrokerSet with partition info " + - s"$updateMetadataRequestPartitionInfoMap might be lost ") - metadataInstance = metadataProvider() - } - - def setUpdateType(updateType: AbstractControlRequest.Type): Unit = { - this.updateType = updateType - } - - def clear(): Unit = { - leaderAndIsrRequestMap.clear() - stopReplicaRequestMap.clear() - updateMetadataRequestBrokerSet.clear() - updateMetadataRequestPartitionInfoMap.clear() - metadataInstance = null - updateType = AbstractControlRequest.Type.UNKNOWN - } - - def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], - topicPartition: TopicPartition, - leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - replicaAssignment: ReplicaAssignment, - isNew: Boolean): Unit = { - - brokerIds.filter(_ >= 0).foreach { brokerId => - val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) - val alreadyNew = result.get(topicPartition).exists(_.isNew) - val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - val partitionState = new LeaderAndIsrPartitionState() - .setTopicName(topicPartition.topic) - .setPartitionIndex(topicPartition.partition) - .setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch) - .setLeader(leaderAndIsr.leader) - .setLeaderEpoch(leaderAndIsr.leaderEpoch) - .setIsr(leaderAndIsr.isr) - .setPartitionEpoch(leaderAndIsr.partitionEpoch) - .setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava) - .setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava) - .setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava) - .setIsNew(isNew || alreadyNew) - - if (metadataVersionProvider.apply().isAtLeast(IBP_3_2_IV0)) { - partitionState.setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) - } - - result.put(topicPartition, partitionState) - } - - addUpdateMetadataRequestForBrokers(metadataInstance.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition)) - } - - def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], - topicPartition: TopicPartition, - deletePartition: Boolean): Unit = { - // A sentinel (-2) is used as an epoch if the topic is queued for deletion. It overrides - // any existing epoch. - val leaderEpoch = metadataInstance.leaderEpoch(topicPartition) - - brokerIds.filter(_ >= 0).foreach { brokerId => - val result = stopReplicaRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) - val alreadyDelete = result.get(topicPartition).exists(_.deletePartition) - result.put(topicPartition, new StopReplicaPartitionState() - .setPartitionIndex(topicPartition.partition()) - .setLeaderEpoch(leaderEpoch) - .setDeletePartition(alreadyDelete || deletePartition)) - } - } - - /** Send UpdateMetadataRequest to the given brokers for the given partitions and partitions that are being deleted */ - def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], - partitions: collection.Set[TopicPartition]): Unit = { - updateMetadataRequestBrokerSet ++= brokerIds.filter(_ >= 0) - partitions.foreach { partition => - val beingDeleted = metadataInstance.isTopicQueuedUpForDeletion(partition.topic()) - metadataInstance.partitionLeadershipInfo(partition) match { - case Some(LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) => - val replicas = metadataInstance.partitionReplicaAssignment(partition) - val offlineReplicas = replicas.filter(!metadataInstance.isReplicaOnline(_, partition)) - val updatedLeaderAndIsr = - if (beingDeleted) LeaderAndIsr.duringDelete(leaderAndIsr.isr) - else leaderAndIsr - addUpdateMetadataRequestForBrokers(brokerIds, controllerEpoch, partition, - updatedLeaderAndIsr.leader, updatedLeaderAndIsr.leaderEpoch, updatedLeaderAndIsr.partitionEpoch, - updatedLeaderAndIsr.isr.asScala.map(_.toInt).toList, replicas, offlineReplicas) - case None => - info(s"Leader not yet assigned for partition $partition. Skip sending UpdateMetadataRequest.") - } - } - } - - def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int]): Unit = { - updateMetadataRequestBrokerSet ++= brokerIds.filter(_ >= 0) - } - - def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], - controllerEpoch: Int, - partition: TopicPartition, - leader: Int, - leaderEpoch: Int, - partitionEpoch: Int, - isrs: List[Int], - replicas: Seq[Int], - offlineReplicas: Seq[Int]): Unit = { - updateMetadataRequestBrokerSet ++= brokerIds.filter(_ >= 0) - val partitionStateInfo = new UpdateMetadataPartitionState() - .setTopicName(partition.topic) - .setPartitionIndex(partition.partition) - .setControllerEpoch(controllerEpoch) - .setLeader(leader) - .setLeaderEpoch(leaderEpoch) - .setIsr(isrs.map(Integer.valueOf).asJava) - .setZkVersion(partitionEpoch) - .setReplicas(replicas.map(Integer.valueOf).asJava) - .setOfflineReplicas(offlineReplicas.map(Integer.valueOf).asJava) - updateMetadataRequestPartitionInfoMap.put(partition, partitionStateInfo) - } - - private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { - val metadataVersion = metadataVersionProvider.apply() - val leaderAndIsrRequestVersion: Short = - if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 7 - else if (metadataVersion.isAtLeast(IBP_3_2_IV0)) 6 - else 5 - - leaderAndIsrRequestMap.foreachEntry { (broker, leaderAndIsrPartitionStates) => - if (metadataInstance.liveOrShuttingDownBrokerIds.contains(broker)) { - val leaderIds = mutable.Set.empty[Int] - var numBecomeLeaders = 0 - leaderAndIsrPartitionStates.foreachEntry { (topicPartition, state) => - leaderIds += state.leader - val typeOfRequest = if (broker == state.leader) { - numBecomeLeaders += 1 - "become-leader" - } else { - "become-follower" - } - if (stateChangeLog.isTraceEnabled) - stateChangeLog.trace(s"Sending $typeOfRequest LeaderAndIsr request $state to broker $broker for partition $topicPartition") - } - stateChangeLog.info(s"Sending LeaderAndIsr request to broker $broker with $numBecomeLeaders become-leader " + - s"and ${leaderAndIsrPartitionStates.size - numBecomeLeaders} become-follower partitions") - val leaders = metadataInstance.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { - _.node(config.interBrokerListenerName) - } - val brokerEpoch = metadataInstance.liveBrokerIdAndEpochs(broker) - val topicIds = leaderAndIsrPartitionStates.keys - .map(_.topic) - .toSet[String] - .map(topic => (topic, metadataInstance.topicIds.getOrElse(topic, Uuid.ZERO_UUID))) - .toMap - val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder( - leaderAndIsrRequestVersion, - controllerId, - controllerEpoch, - brokerEpoch, - leaderAndIsrPartitionStates.values.toBuffer.asJava, - topicIds.asJava, - leaders.asJava, - kraftController, - updateType - ) - sendRequest(broker, leaderAndIsrRequestBuilder, (r: AbstractResponse) => { - val leaderAndIsrResponse = r.asInstanceOf[LeaderAndIsrResponse] - handleLeaderAndIsrResponse(leaderAndIsrResponse, broker) - }) - } - } - leaderAndIsrRequestMap.clear() - } - - def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit - - private def sendUpdateMetadataRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { - stateChangeLog.info(s"Sending UpdateMetadata request to brokers $updateMetadataRequestBrokerSet " + - s"for ${updateMetadataRequestPartitionInfoMap.size} partitions") - - val partitionStates = updateMetadataRequestPartitionInfoMap.values.toBuffer - val metadataVersion = metadataVersionProvider.apply() - val updateMetadataRequestVersion: Short = - if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 8 - else 7 - - val liveBrokers = metadataInstance.liveOrShuttingDownBrokers.iterator.map { broker => - val endpoints = if (updateMetadataRequestVersion == 0) { - // Version 0 of UpdateMetadataRequest only supports PLAINTEXT - val securityProtocol = SecurityProtocol.PLAINTEXT - val listenerName = ListenerName.forSecurityProtocol(securityProtocol) - val node = broker.node(listenerName) - Seq(new UpdateMetadataEndpoint() - .setHost(node.host) - .setPort(node.port) - .setSecurityProtocol(securityProtocol.id) - .setListener(listenerName.value)) - } else { - broker.endPoints.map { endpoint => - new UpdateMetadataEndpoint() - .setHost(endpoint.host) - .setPort(endpoint.port) - .setSecurityProtocol(endpoint.securityProtocol.id) - .setListener(endpoint.listenerName.value) - } - } - new UpdateMetadataBroker() - .setId(broker.id) - .setEndpoints(endpoints.asJava) - .setRack(broker.rack.orNull) - }.toBuffer - - updateMetadataRequestBrokerSet.intersect(metadataInstance.liveOrShuttingDownBrokerIds).foreach { broker => - val brokerEpoch = metadataInstance.liveBrokerIdAndEpochs(broker) - val topicIds = partitionStates.map(_.topicName()) - .distinct - .filter(metadataInstance.topicIds.contains) - .map(topic => (topic, metadataInstance.topicIds(topic))).toMap - val updateMetadataRequestBuilder = new UpdateMetadataRequest.Builder( - updateMetadataRequestVersion, - controllerId, - controllerEpoch, - brokerEpoch, - partitionStates.asJava, - liveBrokers.asJava, - topicIds.asJava, - kraftController, - updateType - ) - sendRequest(broker, updateMetadataRequestBuilder, (r: AbstractResponse) => { - val updateMetadataResponse = r.asInstanceOf[UpdateMetadataResponse] - handleUpdateMetadataResponse(updateMetadataResponse, broker) - }) - - } - updateMetadataRequestBrokerSet.clear() - updateMetadataRequestPartitionInfoMap.clear() - } - - def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int): Unit - - private def sendStopReplicaRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { - val traceEnabled = stateChangeLog.isTraceEnabled - val metadataVersion = metadataVersionProvider.apply() - val stopReplicaRequestVersion: Short = - if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 4 - else 3 - - def responseCallback(brokerId: Int, isPartitionDeleted: TopicPartition => Boolean) - (response: AbstractResponse): Unit = { - val stopReplicaResponse = response.asInstanceOf[StopReplicaResponse] - val partitionErrorsForDeletingTopics = mutable.Map.empty[TopicPartition, Errors] - stopReplicaResponse.partitionErrors.forEach { pe => - val tp = new TopicPartition(pe.topicName, pe.partitionIndex) - if (metadataInstance.isTopicDeletionInProgress(pe.topicName) && - isPartitionDeleted(tp)) { - partitionErrorsForDeletingTopics += tp -> Errors.forCode(pe.errorCode) - } - } - if (partitionErrorsForDeletingTopics.nonEmpty) - handleStopReplicaResponse(stopReplicaResponse, brokerId, partitionErrorsForDeletingTopics.toMap) - } - - stopReplicaRequestMap.foreachEntry { (brokerId, partitionStates) => - if (metadataInstance.liveOrShuttingDownBrokerIds.contains(brokerId)) { - if (traceEnabled) - partitionStates.foreachEntry { (topicPartition, partitionState) => - stateChangeLog.trace(s"Sending StopReplica request $partitionState to " + - s"broker $brokerId for partition $topicPartition") - } - - val brokerEpoch = metadataInstance.liveBrokerIdAndEpochs(brokerId) - if (stopReplicaRequestVersion >= 3) { - val stopReplicaTopicState = mutable.Map.empty[String, StopReplicaTopicState] - partitionStates.foreachEntry { (topicPartition, partitionState) => - val topicState = stopReplicaTopicState.getOrElseUpdate(topicPartition.topic, - new StopReplicaTopicState().setTopicName(topicPartition.topic)) - topicState.partitionStates().add(partitionState) - } - - stateChangeLog.info(s"Sending StopReplica request for ${partitionStates.size} " + - s"replicas to broker $brokerId") - val stopReplicaRequestBuilder = new StopReplicaRequest.Builder( - stopReplicaRequestVersion, controllerId, controllerEpoch, brokerEpoch, - false, stopReplicaTopicState.values.toBuffer.asJava, kraftController) - sendRequest(brokerId, stopReplicaRequestBuilder, - responseCallback(brokerId, tp => partitionStates.get(tp).exists(_.deletePartition))) - } else { - var numPartitionStateWithDelete = 0 - var numPartitionStateWithoutDelete = 0 - val topicStatesWithDelete = mutable.Map.empty[String, StopReplicaTopicState] - val topicStatesWithoutDelete = mutable.Map.empty[String, StopReplicaTopicState] - - partitionStates.foreachEntry { (topicPartition, partitionState) => - val topicStates = if (partitionState.deletePartition()) { - numPartitionStateWithDelete += 1 - topicStatesWithDelete - } else { - numPartitionStateWithoutDelete += 1 - topicStatesWithoutDelete - } - val topicState = topicStates.getOrElseUpdate(topicPartition.topic, - new StopReplicaTopicState().setTopicName(topicPartition.topic)) - topicState.partitionStates().add(partitionState) - } - - if (topicStatesWithDelete.nonEmpty) { - stateChangeLog.info(s"Sending StopReplica request (delete = true) for " + - s"$numPartitionStateWithDelete replicas to broker $brokerId") - val stopReplicaRequestBuilder = new StopReplicaRequest.Builder( - stopReplicaRequestVersion, controllerId, controllerEpoch, brokerEpoch, - true, topicStatesWithDelete.values.toBuffer.asJava, kraftController) - sendRequest(brokerId, stopReplicaRequestBuilder, responseCallback(brokerId, _ => true)) - } - - if (topicStatesWithoutDelete.nonEmpty) { - stateChangeLog.info(s"Sending StopReplica request (delete = false) for " + - s"$numPartitionStateWithoutDelete replicas to broker $brokerId") - val stopReplicaRequestBuilder = new StopReplicaRequest.Builder( - stopReplicaRequestVersion, controllerId, controllerEpoch, brokerEpoch, - false, topicStatesWithoutDelete.values.toBuffer.asJava, kraftController) - sendRequest(brokerId, stopReplicaRequestBuilder) - } - } - } - } - - stopReplicaRequestMap.clear() - } - - def handleStopReplicaResponse(stopReplicaResponse: StopReplicaResponse, brokerId: Int, - partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit - - def sendRequestsToBrokers(controllerEpoch: Int): Unit = { - try { - val stateChangeLog = stateChangeLogger.withControllerEpoch(controllerEpoch) - sendLeaderAndIsrRequest(controllerEpoch, stateChangeLog) - sendUpdateMetadataRequests(controllerEpoch, stateChangeLog) - sendStopReplicaRequests(controllerEpoch, stateChangeLog) - this.updateType = AbstractControlRequest.Type.UNKNOWN - } catch { - case e: Throwable => - if (leaderAndIsrRequestMap.nonEmpty) { - error("Haven't been able to send leader and isr requests, current state of " + - s"the map is $leaderAndIsrRequestMap. Exception message: $e") - } - if (updateMetadataRequestBrokerSet.nonEmpty) { - error(s"Haven't been able to send metadata update requests to brokers $updateMetadataRequestBrokerSet, " + - s"current state of the partition info is $updateMetadataRequestPartitionInfoMap. Exception message: $e") - } - if (stopReplicaRequestMap.nonEmpty) { - error("Haven't been able to send stop replica requests, current state of " + - s"the map is $stopReplicaRequestMap. Exception message: $e") - } - throw new IllegalStateException(e) - } - } -} - -case class ControllerBrokerStateInfo(networkClient: NetworkClient, - brokerNode: Node, - messageQueue: BlockingQueue[QueueItem], - requestSendThread: RequestSendThread, - queueSizeGauge: Gauge[Int], - requestRateAndTimeMetrics: Timer, - reconfigurableChannelBuilder: Option[Reconfigurable]) - diff --git a/core/src/main/scala/kafka/controller/ControllerContext.scala b/core/src/main/scala/kafka/controller/ControllerContext.scala index 1042071641607..cd56510e9a818 100644 --- a/core/src/main/scala/kafka/controller/ControllerContext.scala +++ b/core/src/main/scala/kafka/controller/ControllerContext.scala @@ -17,11 +17,7 @@ package kafka.controller -import kafka.cluster.Broker -import org.apache.kafka.common.{TopicPartition, Uuid} -import org.apache.kafka.metadata.LeaderAndIsr - -import scala.collection.{Map, Seq, Set, mutable} +import scala.collection.Seq object ReplicaAssignment { def apply(replicas: Seq[Int]): ReplicaAssignment = { @@ -42,506 +38,15 @@ case class ReplicaAssignment private (replicas: Seq[Int], addingReplicas: Seq[Int], removingReplicas: Seq[Int]) { - lazy val originReplicas: Seq[Int] = replicas.diff(addingReplicas) lazy val targetReplicas: Seq[Int] = replicas.diff(removingReplicas) def isBeingReassigned: Boolean = { addingReplicas.nonEmpty || removingReplicas.nonEmpty } - def reassignTo(target: Seq[Int]): ReplicaAssignment = { - val fullReplicaSet = (target ++ originReplicas).distinct - ReplicaAssignment( - fullReplicaSet, - fullReplicaSet.diff(originReplicas), - fullReplicaSet.diff(target) - ) - } - - def removeReplica(replica: Int): ReplicaAssignment = { - ReplicaAssignment( - replicas.filterNot(_ == replica), - addingReplicas.filterNot(_ == replica), - removingReplicas.filterNot(_ == replica) - ) - } - override def toString: String = s"ReplicaAssignment(" + s"replicas=${replicas.mkString(",")}, " + s"addingReplicas=${addingReplicas.mkString(",")}, " + s"removingReplicas=${removingReplicas.mkString(",")})" } -class ControllerContext extends ControllerChannelContext { - val stats = new ControllerStats - var offlinePartitionCount = 0 - var preferredReplicaImbalanceCount = 0 - val shuttingDownBrokerIds = mutable.Set.empty[Int] - private val liveBrokers = mutable.Set.empty[Broker] - private val liveBrokerEpochs = mutable.Map.empty[Int, Long] - var epoch: Int = KafkaController.InitialControllerEpoch - var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion - - val allTopics = mutable.Set.empty[String] - var topicIds = mutable.Map.empty[String, Uuid] - var topicNames = mutable.Map.empty[Uuid, String] - val partitionAssignments = mutable.Map.empty[String, mutable.Map[Int, ReplicaAssignment]] - private val partitionLeadershipInfo = mutable.Map.empty[TopicPartition, LeaderIsrAndControllerEpoch] - val partitionsBeingReassigned = mutable.Set.empty[TopicPartition] - val partitionStates = mutable.Map.empty[TopicPartition, PartitionState] - val replicaStates = mutable.Map.empty[PartitionAndReplica, ReplicaState] - val replicasOnOfflineDirs = mutable.Map.empty[Int, Set[TopicPartition]] - - val topicsToBeDeleted = mutable.Set.empty[String] - - /** The following topicsWithDeletionStarted variable is used to properly update the offlinePartitionCount metric. - * When a topic is going through deletion, we don't want to keep track of its partition state - * changes in the offlinePartitionCount metric. This goal means if some partitions of a topic are already - * in OfflinePartition state when deletion starts, we need to change the corresponding partition - * states to NonExistentPartition first before starting the deletion. - * - * However we can NOT change partition states to NonExistentPartition at the time of enqueuing topics - * for deletion. The reason is that when a topic is enqueued for deletion, it may be ineligible for - * deletion due to ongoing partition reassignments. Hence there might be a delay between enqueuing - * a topic for deletion and the actual start of deletion. In this delayed interval, partitions may still - * transition to or out of the OfflinePartition state. - * - * Hence we decide to change partition states to NonExistentPartition only when the actual deletion have started. - * For topics whose deletion have actually started, we keep track of them in the following topicsWithDeletionStarted - * variable. And once a topic is in the topicsWithDeletionStarted set, we are sure there will no longer - * be partition reassignments to any of its partitions, and only then it's safe to move its partitions to - * NonExistentPartition state. Once a topic is in the topicsWithDeletionStarted set, we will stop monitoring - * its partition state changes in the offlinePartitionCount metric - */ - val topicsWithDeletionStarted = mutable.Set.empty[String] - val topicsIneligibleForDeletion = mutable.Set.empty[String] - - private def clearTopicsState(): Unit = { - allTopics.clear() - topicIds.clear() - topicNames.clear() - partitionAssignments.clear() - partitionLeadershipInfo.clear() - partitionsBeingReassigned.clear() - replicasOnOfflineDirs.clear() - partitionStates.clear() - offlinePartitionCount = 0 - preferredReplicaImbalanceCount = 0 - replicaStates.clear() - } - - def addTopicId(topic: String, id: Uuid): Unit = { - if (!allTopics.contains(topic)) - throw new IllegalStateException(s"topic $topic is not contained in all topics.") - - topicIds.get(topic).foreach { existingId => - if (!existingId.equals(id)) - throw new IllegalStateException(s"topic ID map already contained ID for topic " + - s"$topic and new ID $id did not match existing ID $existingId") - } - topicNames.get(id).foreach { existingName => - if (!existingName.equals(topic)) - throw new IllegalStateException(s"topic name map already contained ID " + - s"$id and new name $topic did not match existing name $existingName") - } - topicIds.put(topic, id) - topicNames.put(id, topic) - } - - def partitionReplicaAssignment(topicPartition: TopicPartition): Seq[Int] = { - partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty).get(topicPartition.partition) match { - case Some(partitionAssignment) => partitionAssignment.replicas - case None => Seq.empty - } - } - - def partitionFullReplicaAssignment(topicPartition: TopicPartition): ReplicaAssignment = { - partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty) - .getOrElse(topicPartition.partition, ReplicaAssignment.empty) - } - - def updatePartitionFullReplicaAssignment(topicPartition: TopicPartition, newAssignment: ReplicaAssignment): Unit = { - val assignments = partitionAssignments.getOrElseUpdate(topicPartition.topic, mutable.Map.empty) - val previous = assignments.put(topicPartition.partition, newAssignment) - val leadershipInfo = partitionLeadershipInfo.get(topicPartition) - updatePreferredReplicaImbalanceMetric(topicPartition, previous, leadershipInfo, - Some(newAssignment), leadershipInfo) - } - - def partitionReplicaAssignmentForTopic(topic : String): Map[TopicPartition, Seq[Int]] = { - partitionAssignments.getOrElse(topic, Map.empty).map { - case (partition, assignment) => (new TopicPartition(topic, partition), assignment.replicas) - }.toMap - } - - def partitionFullReplicaAssignmentForTopic(topic : String): Map[TopicPartition, ReplicaAssignment] = { - partitionAssignments.getOrElse(topic, Map.empty).map { - case (partition, assignment) => (new TopicPartition(topic, partition), assignment) - }.toMap - } - - def allPartitions: Set[TopicPartition] = { - partitionAssignments.flatMap { - case (topic, topicReplicaAssignment) => topicReplicaAssignment.map { - case (partition, _) => new TopicPartition(topic, partition) - } - }.toSet - } - - def setLiveBrokers(brokerAndEpochs: Map[Broker, Long]): Unit = { - clearLiveBrokers() - addLiveBrokers(brokerAndEpochs) - } - - private def clearLiveBrokers(): Unit = { - liveBrokers.clear() - liveBrokerEpochs.clear() - } - - def addLiveBrokers(brokerAndEpochs: Map[Broker, Long]): Unit = { - liveBrokers ++= brokerAndEpochs.keySet - liveBrokerEpochs ++= brokerAndEpochs.map { case (broker, brokerEpoch) => (broker.id, brokerEpoch) } - } - - def removeLiveBrokers(brokerIds: Set[Int]): Unit = { - liveBrokers --= liveBrokers.filter(broker => brokerIds.contains(broker.id)) - liveBrokerEpochs --= brokerIds - } - - def updateBrokerMetadata(oldMetadata: Broker, newMetadata: Broker): Unit = { - liveBrokers -= oldMetadata - liveBrokers += newMetadata - } - - // getter - def liveBrokerIds: Set[Int] = liveBrokerEpochs.keySet.diff(shuttingDownBrokerIds) - // To just check if a broker is live, we should use this method instead of liveBrokerIds.contains(brokerId) - // which is more expensive because it constructs the set of live broker IDs. - // See KAFKA-17061 for the details. - def isLiveBroker(brokerId: Int): Boolean = liveBrokerEpochs.contains(brokerId) && !shuttingDownBrokerIds(brokerId) - def liveOrShuttingDownBrokerIds: Set[Int] = liveBrokerEpochs.keySet - def liveOrShuttingDownBrokers: Set[Broker] = liveBrokers - def liveBrokerIdAndEpochs: Map[Int, Long] = liveBrokerEpochs - def liveOrShuttingDownBroker(brokerId: Int): Option[Broker] = liveOrShuttingDownBrokers.find(_.id == brokerId) - - def partitionsOnBroker(brokerId: Int): Set[TopicPartition] = { - partitionAssignments.flatMap { - case (topic, topicReplicaAssignment) => topicReplicaAssignment.filter { - case (_, partitionAssignment) => partitionAssignment.replicas.contains(brokerId) - }.map { - case (partition, _) => new TopicPartition(topic, partition) - } - }.toSet - } - - def isReplicaOnline(brokerId: Int, topicPartition: TopicPartition): Boolean = { - isReplicaOnline(brokerId, topicPartition, includeShuttingDownBrokers = false) - } - - def isReplicaOnline(brokerId: Int, topicPartition: TopicPartition, includeShuttingDownBrokers: Boolean): Boolean = { - val brokerOnline = { - if (includeShuttingDownBrokers) liveOrShuttingDownBrokerIds.contains(brokerId) - else isLiveBroker(brokerId) - } - brokerOnline && !replicasOnOfflineDirs.getOrElse(brokerId, Set.empty).contains(topicPartition) - } - - def replicasOnBrokers(brokerIds: Set[Int]): Set[PartitionAndReplica] = { - brokerIds.flatMap { brokerId => - partitionAssignments.flatMap { - case (topic, topicReplicaAssignment) => topicReplicaAssignment.collect { - case (partition, partitionAssignment) if partitionAssignment.replicas.contains(brokerId) => - PartitionAndReplica(new TopicPartition(topic, partition), brokerId) - } - } - } - } - - def replicasForTopic(topic: String): Set[PartitionAndReplica] = { - partitionAssignments.getOrElse(topic, mutable.Map.empty).flatMap { - case (partition, assignment) => assignment.replicas.map { r => - PartitionAndReplica(new TopicPartition(topic, partition), r) - } - }.toSet - } - - def partitionsForTopic(topic: String): collection.Set[TopicPartition] = { - partitionAssignments.getOrElse(topic, mutable.Map.empty).map { - case (partition, _) => new TopicPartition(topic, partition) - }.toSet - } - - /** - * Get all online and offline replicas. - * - * @return a tuple consisting of first the online replicas and followed by the offline replicas - */ - def onlineAndOfflineReplicas: (Set[PartitionAndReplica], Set[PartitionAndReplica]) = { - val onlineReplicas = mutable.Set.empty[PartitionAndReplica] - val offlineReplicas = mutable.Set.empty[PartitionAndReplica] - for ((topic, partitionAssignments) <- partitionAssignments; - (partitionId, assignment) <- partitionAssignments) { - val partition = new TopicPartition(topic, partitionId) - for (replica <- assignment.replicas) { - val partitionAndReplica = PartitionAndReplica(partition, replica) - if (isReplicaOnline(replica, partition)) - onlineReplicas.add(partitionAndReplica) - else - offlineReplicas.add(partitionAndReplica) - } - } - (onlineReplicas, offlineReplicas) - } - - def replicasForPartition(partitions: collection.Set[TopicPartition]): collection.Set[PartitionAndReplica] = { - partitions.flatMap { p => - val replicas = partitionReplicaAssignment(p) - replicas.map(PartitionAndReplica(p, _)) - } - } - - def resetContext(): Unit = { - topicsToBeDeleted.clear() - topicsWithDeletionStarted.clear() - topicsIneligibleForDeletion.clear() - shuttingDownBrokerIds.clear() - epoch = 0 - epochZkVersion = 0 - clearTopicsState() - clearLiveBrokers() - } - - def setAllTopics(topics: Set[String]): Unit = { - allTopics.clear() - allTopics ++= topics - } - - def removeTopic(topic: String): Unit = { - // Metric is cleaned when the topic is queued up for deletion so - // we don't clean it twice. We clean it only if it is deleted - // directly. - if (!topicsToBeDeleted.contains(topic)) - cleanPreferredReplicaImbalanceMetric(topic) - topicsToBeDeleted -= topic - topicsWithDeletionStarted -= topic - allTopics -= topic - topicIds.remove(topic).foreach { topicId => - topicNames.remove(topicId) - } - partitionAssignments.remove(topic).foreach { assignments => - assignments.keys.foreach { partition => - partitionLeadershipInfo.remove(new TopicPartition(topic, partition)) - } - } - } - - def queueTopicDeletion(topicToBeAddedIntoDeletionList: Set[String]): Unit = { - // queueTopicDeletion could be called multiple times for same topic. - // e.g. 1) delete topic-A => 2) delete topic-B before A's deletion completes. - // In this case, at 2), queueTopicDeletion will be called with Set(topic-A, topic-B). - // However we should call cleanPreferredReplicaImbalanceMetric only once for same topic - // because otherwise, preferredReplicaImbalanceCount could be decremented wrongly at 2nd call. - // So we need to take a diff with already queued topics here. - val newlyDeletedTopics = topicToBeAddedIntoDeletionList.diff(topicsToBeDeleted) - topicsToBeDeleted ++= newlyDeletedTopics - newlyDeletedTopics.foreach(cleanPreferredReplicaImbalanceMetric) - } - - def beginTopicDeletion(topics: Set[String]): Unit = { - topicsWithDeletionStarted ++= topics - } - - def isTopicDeletionInProgress(topic: String): Boolean = { - topicsWithDeletionStarted.contains(topic) - } - - def isTopicQueuedUpForDeletion(topic: String): Boolean = { - topicsToBeDeleted.contains(topic) - } - - def isTopicEligibleForDeletion(topic: String): Boolean = { - topicsToBeDeleted.contains(topic) && !topicsIneligibleForDeletion.contains(topic) - } - - def topicsQueuedForDeletion: Set[String] = { - topicsToBeDeleted - } - - def replicasInState(topic: String, state: ReplicaState): Set[PartitionAndReplica] = { - replicasForTopic(topic).filter(replica => replicaStates(replica) == state).toSet - } - - def areAllReplicasInState(topic: String, state: ReplicaState): Boolean = { - replicasForTopic(topic).forall(replica => replicaStates(replica) == state) - } - - def isAnyReplicaInState(topic: String, state: ReplicaState): Boolean = { - replicasForTopic(topic).exists(replica => replicaStates(replica) == state) - } - - def checkValidReplicaStateChange(replicas: Seq[PartitionAndReplica], targetState: ReplicaState): (Seq[PartitionAndReplica], Seq[PartitionAndReplica]) = { - replicas.partition(replica => isValidReplicaStateTransition(replica, targetState)) - } - - def checkValidPartitionStateChange(partitions: Seq[TopicPartition], targetState: PartitionState): (Seq[TopicPartition], Seq[TopicPartition]) = { - partitions.partition(p => isValidPartitionStateTransition(p, targetState)) - } - - def putReplicaState(replica: PartitionAndReplica, state: ReplicaState): Unit = { - replicaStates.put(replica, state) - } - - def removeReplicaState(replica: PartitionAndReplica): Unit = { - replicaStates.remove(replica) - } - - def putReplicaStateIfNotExists(replica: PartitionAndReplica, state: ReplicaState): Unit = { - replicaStates.getOrElseUpdate(replica, state) - } - - def putPartitionState(partition: TopicPartition, targetState: PartitionState): Unit = { - val currentState = partitionStates.put(partition, targetState).getOrElse(NonExistentPartition) - updatePartitionStateMetrics(partition, currentState, targetState) - } - - private def updatePartitionStateMetrics(partition: TopicPartition, - currentState: PartitionState, - targetState: PartitionState): Unit = { - if (!isTopicDeletionInProgress(partition.topic)) { - if (currentState != OfflinePartition && targetState == OfflinePartition) { - offlinePartitionCount = offlinePartitionCount + 1 - } else if (currentState == OfflinePartition && targetState != OfflinePartition) { - offlinePartitionCount = offlinePartitionCount - 1 - } - } - } - - def putPartitionStateIfNotExists(partition: TopicPartition, state: PartitionState): Unit = { - if (partitionStates.getOrElseUpdate(partition, state) == state) - updatePartitionStateMetrics(partition, NonExistentPartition, state) - } - - def replicaState(replica: PartitionAndReplica): ReplicaState = { - replicaStates(replica) - } - - def partitionState(partition: TopicPartition): PartitionState = { - partitionStates(partition) - } - - def partitionsInState(state: PartitionState): Set[TopicPartition] = { - partitionStates.filter { case (_, s) => s == state }.keySet.toSet - } - - def partitionsInStates(states: Set[PartitionState]): Set[TopicPartition] = { - partitionStates.filter { case (_, s) => states.contains(s) }.keySet.toSet - } - - def partitionsInState(topic: String, state: PartitionState): Set[TopicPartition] = { - partitionsForTopic(topic).filter { partition => state == partitionState(partition) }.toSet - } - - def partitionsInStates(topic: String, states: Set[PartitionState]): Set[TopicPartition] = { - partitionsForTopic(topic).filter { partition => states.contains(partitionState(partition)) }.toSet - } - - def putPartitionLeadershipInfo(partition: TopicPartition, - leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch): Unit = { - val previous = partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch) - val replicaAssignment = partitionFullReplicaAssignment(partition) - updatePreferredReplicaImbalanceMetric(partition, Some(replicaAssignment), previous, - Some(replicaAssignment), Some(leaderIsrAndControllerEpoch)) - } - - def leaderEpoch(partition: TopicPartition): Int = { - // A sentinel (-2) is used as an epoch if the topic is queued for deletion. It overrides - // any existing epoch. - if (isTopicQueuedUpForDeletion(partition.topic)) { - LeaderAndIsr.EPOCH_DURING_DELETE - } else { - partitionLeadershipInfo.get(partition) - .map(_.leaderAndIsr.leaderEpoch) - .getOrElse(LeaderAndIsr.NO_EPOCH) - } - } - - def partitionLeadershipInfo(partition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { - partitionLeadershipInfo.get(partition) - } - - def partitionsLeadershipInfo: Map[TopicPartition, LeaderIsrAndControllerEpoch] = - partitionLeadershipInfo - - def partitionsWithLeaders: Set[TopicPartition] = - partitionLeadershipInfo.keySet.filter(tp => !isTopicQueuedUpForDeletion(tp.topic)) - - def partitionsWithOfflineLeader: Set[TopicPartition] = { - partitionLeadershipInfo.filter { case (topicPartition, leaderIsrAndControllerEpoch) => - !isReplicaOnline(leaderIsrAndControllerEpoch.leaderAndIsr.leader, topicPartition) && - !isTopicQueuedUpForDeletion(topicPartition.topic) - }.keySet - } - - def partitionLeadersOnBroker(brokerId: Int): Set[TopicPartition] = { - partitionLeadershipInfo.filter { case (topicPartition, leaderIsrAndControllerEpoch) => - !isTopicQueuedUpForDeletion(topicPartition.topic) && - leaderIsrAndControllerEpoch.leaderAndIsr.leader == brokerId && - partitionReplicaAssignment(topicPartition).size > 1 - }.keySet - } - - def topicName(topicId: Uuid): Option[String] = { - topicNames.get(topicId) - } - - def clearPartitionLeadershipInfo(): Unit = partitionLeadershipInfo.clear() - - def partitionWithLeadersCount: Int = partitionLeadershipInfo.size - - private def updatePreferredReplicaImbalanceMetric(partition: TopicPartition, - oldReplicaAssignment: Option[ReplicaAssignment], - oldLeadershipInfo: Option[LeaderIsrAndControllerEpoch], - newReplicaAssignment: Option[ReplicaAssignment], - newLeadershipInfo: Option[LeaderIsrAndControllerEpoch]): Unit = { - if (!isTopicQueuedUpForDeletion(partition.topic)) { - oldReplicaAssignment.foreach { replicaAssignment => - oldLeadershipInfo.foreach { leadershipInfo => - if (!hasPreferredLeader(replicaAssignment, leadershipInfo)) - preferredReplicaImbalanceCount -= 1 - } - } - - newReplicaAssignment.foreach { replicaAssignment => - newLeadershipInfo.foreach { leadershipInfo => - if (!hasPreferredLeader(replicaAssignment, leadershipInfo)) - preferredReplicaImbalanceCount += 1 - } - } - } - } - - private def cleanPreferredReplicaImbalanceMetric(topic: String): Unit = { - partitionAssignments.getOrElse(topic, mutable.Map.empty).foreachEntry { (partition, replicaAssignment) => - partitionLeadershipInfo.get(new TopicPartition(topic, partition)).foreach { leadershipInfo => - if (!hasPreferredLeader(replicaAssignment, leadershipInfo)) - preferredReplicaImbalanceCount -= 1 - } - } - } - - private def hasPreferredLeader(replicaAssignment: ReplicaAssignment, - leadershipInfo: LeaderIsrAndControllerEpoch): Boolean = { - val preferredReplica = replicaAssignment.replicas.head - if (replicaAssignment.isBeingReassigned && replicaAssignment.addingReplicas.contains(preferredReplica)) - // reassigning partitions are not counted as imbalanced until the new replica joins the ISR (completes reassignment) - !leadershipInfo.leaderAndIsr.isr.contains(preferredReplica) - else - leadershipInfo.leaderAndIsr.leader == preferredReplica - } - - private def isValidReplicaStateTransition(replica: PartitionAndReplica, targetState: ReplicaState): Boolean = - targetState.validPreviousStates.contains(replicaStates(replica)) - - private def isValidPartitionStateTransition(partition: TopicPartition, targetState: PartitionState): Boolean = - targetState.validPreviousStates.contains(partitionStates(partition)) -} diff --git a/core/src/main/scala/kafka/controller/ControllerEventManager.scala b/core/src/main/scala/kafka/controller/ControllerEventManager.scala deleted file mode 100644 index f4ea593f15e84..0000000000000 --- a/core/src/main/scala/kafka/controller/ControllerEventManager.scala +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.controller - -import com.yammer.metrics.core.Timer - -import java.util -import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit} -import java.util.concurrent.locks.ReentrantLock -import kafka.utils.CoreUtils.inLock -import kafka.utils.Logging -import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.metrics.KafkaMetricsGroup -import org.apache.kafka.server.util.ShutdownableThread - -import scala.collection._ - -object ControllerEventManager { - val ControllerEventThreadName = "controller-event-thread" - private val EventQueueTimeMetricName = "EventQueueTimeMs" - private val EventQueueSizeMetricName = "EventQueueSize" -} - -trait ControllerEventProcessor { - def process(event: ControllerEvent): Unit - def preempt(event: ControllerEvent): Unit -} - -class QueuedEvent(val event: ControllerEvent, - val enqueueTimeMs: Long) { - private val processingStarted = new CountDownLatch(1) - private val spent = new AtomicBoolean(false) - - def process(processor: ControllerEventProcessor): Unit = { - if (spent.getAndSet(true)) - return - processingStarted.countDown() - processor.process(event) - } - - def preempt(processor: ControllerEventProcessor): Unit = { - if (spent.getAndSet(true)) - return - processor.preempt(event) - } - - def awaitProcessing(): Unit = { - processingStarted.await() - } - - override def toString: String = { - s"QueuedEvent(event=$event, enqueueTimeMs=$enqueueTimeMs)" - } -} - -class ControllerEventManager(controllerId: Int, - processor: ControllerEventProcessor, - time: Time, - rateAndTimeMetrics: Map[ControllerState, Timer], - eventQueueTimeTimeoutMs: Long = 300000) { - import ControllerEventManager._ - - private val metricsGroup = new KafkaMetricsGroup(this.getClass) - - @volatile private var _state: ControllerState = ControllerState.Idle - private val putLock = new ReentrantLock() - private val queue = new LinkedBlockingQueue[QueuedEvent] - // Visible for test - private[controller] var thread = new ControllerEventThread(ControllerEventThreadName) - - private val eventQueueTimeHist = metricsGroup.newHistogram(EventQueueTimeMetricName) - - metricsGroup.newGauge(EventQueueSizeMetricName, () => queue.size) - - def state: ControllerState = _state - - def start(): Unit = thread.start() - - def close(): Unit = { - try { - thread.initiateShutdown() - clearAndPut(ShutdownEventThread) - thread.awaitShutdown() - } finally { - metricsGroup.removeMetric(EventQueueTimeMetricName) - metricsGroup.removeMetric(EventQueueSizeMetricName) - } - } - - def put(event: ControllerEvent): QueuedEvent = inLock(putLock) { - val queuedEvent = new QueuedEvent(event, time.milliseconds()) - queue.put(queuedEvent) - queuedEvent - } - - def clearAndPut(event: ControllerEvent): QueuedEvent = inLock(putLock) { - val preemptedEvents = new util.ArrayList[QueuedEvent]() - queue.drainTo(preemptedEvents) - preemptedEvents.forEach(_.preempt(processor)) - put(event) - } - - def isEmpty: Boolean = queue.isEmpty - - class ControllerEventThread(name: String) - extends ShutdownableThread( - name, false, s"[ControllerEventThread controllerId=$controllerId] ") - with Logging { - - logIdent = logPrefix - - override def doWork(): Unit = { - val dequeued = pollFromEventQueue() - dequeued.event match { - case ShutdownEventThread => // The shutting down of the thread has been initiated at this point. Ignore this event. - case controllerEvent => - _state = controllerEvent.state - - eventQueueTimeHist.update(time.milliseconds() - dequeued.enqueueTimeMs) - - try { - def process(): Unit = dequeued.process(processor) - - rateAndTimeMetrics.get(state) match { - case Some(timer) => timer.time(() => process()) - case None => process() - } - } catch { - case e: Throwable => error(s"Uncaught error processing event $controllerEvent", e) - } - - _state = ControllerState.Idle - } - } - } - - private def pollFromEventQueue(): QueuedEvent = { - val count = eventQueueTimeHist.count() - if (count != 0) { - val event = queue.poll(eventQueueTimeTimeoutMs, TimeUnit.MILLISECONDS) - if (event == null) { - eventQueueTimeHist.clear() - queue.take() - } else { - event - } - } else { - queue.take() - } - } - -} diff --git a/core/src/main/scala/kafka/controller/ControllerState.scala b/core/src/main/scala/kafka/controller/ControllerState.scala deleted file mode 100644 index 0474f6362db43..0000000000000 --- a/core/src/main/scala/kafka/controller/ControllerState.scala +++ /dev/null @@ -1,122 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.controller - -import scala.collection.Seq - -sealed abstract class ControllerState { - - def value: Byte - - def rateAndTimeMetricName: Option[String] = - if (hasRateAndTimeMetric) Some(s"${toString}RateAndTimeMs") else None - - protected def hasRateAndTimeMetric: Boolean = true -} - -object ControllerState { - - // Note: `rateAndTimeMetricName` is based on the case object name by default. Changing a name is a breaking change - // unless `rateAndTimeMetricName` is overridden. - - case object Idle extends ControllerState { - def value = 0 - override protected def hasRateAndTimeMetric: Boolean = false - } - - case object ControllerChange extends ControllerState { - def value = 1 - } - - case object BrokerChange extends ControllerState { - def value = 2 - // The LeaderElectionRateAndTimeMs metric existed before `ControllerState` was introduced and we keep the name - // for backwards compatibility. The alternative would be to have the same metric under two different names. - override def rateAndTimeMetricName: Option[String] = Some("LeaderElectionRateAndTimeMs") - } - - case object TopicChange extends ControllerState { - def value = 3 - } - - case object TopicDeletion extends ControllerState { - def value = 4 - } - - case object AlterPartitionReassignment extends ControllerState { - def value = 5 - - override def rateAndTimeMetricName: Option[String] = Some("PartitionReassignmentRateAndTimeMs") - } - - case object AutoLeaderBalance extends ControllerState { - def value = 6 - } - - case object ManualLeaderBalance extends ControllerState { - def value = 7 - } - - case object ControlledShutdown extends ControllerState { - def value = 8 - } - - case object IsrChange extends ControllerState { - def value = 9 - } - - case object LeaderAndIsrResponseReceived extends ControllerState { - def value = 10 - } - - case object LogDirChange extends ControllerState { - def value = 11 - } - - case object ControllerShutdown extends ControllerState { - def value = 12 - } - - case object UncleanLeaderElectionEnable extends ControllerState { - def value = 13 - } - - case object TopicUncleanLeaderElectionEnable extends ControllerState { - def value = 14 - } - - case object ListPartitionReassignment extends ControllerState { - def value = 15 - } - - case object UpdateMetadataResponseReceived extends ControllerState { - def value = 16 - - override protected def hasRateAndTimeMetric: Boolean = false - } - - case object UpdateFeatures extends ControllerState { - def value = 17 - } - - val values: Seq[ControllerState] = Seq(Idle, ControllerChange, BrokerChange, TopicChange, TopicDeletion, - AlterPartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, - LeaderAndIsrResponseReceived, LogDirChange, ControllerShutdown, UncleanLeaderElectionEnable, - TopicUncleanLeaderElectionEnable, ListPartitionReassignment, UpdateMetadataResponseReceived, - UpdateFeatures) -} diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala deleted file mode 100644 index 973a2ec648242..0000000000000 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ /dev/null @@ -1,2857 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.controller - -import com.yammer.metrics.core.{Meter, Timer} - -import java.util.concurrent.TimeUnit -import kafka.common._ -import kafka.cluster.Broker -import kafka.controller.KafkaController.{ActiveBrokerCountMetricName, ActiveControllerCountMetricName, AlterReassignmentsCallback, ControllerStateMetricName, ElectLeadersCallback, FencedBrokerCountMetricName, GlobalPartitionCountMetricName, GlobalTopicCountMetricName, ListReassignmentsCallback, OfflinePartitionsCountMetricName, PreferredReplicaImbalanceCountMetricName, ReplicasIneligibleToDeleteCountMetricName, ReplicasToDeleteCountMetricName, TopicsIneligibleToDeleteCountMetricName, TopicsToDeleteCountMetricName, UpdateFeaturesCallback} -import kafka.server._ -import kafka.server.metadata.ZkFinalizedFeatureCache -import kafka.utils._ -import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult -import kafka.zk.TopicZNode.TopicIdReplicaAssignment -import kafka.zk.{FeatureZNodeStatus, _} -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.TopicPartition -import org.apache.kafka.common.Uuid -import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterPartitionRequestData, AlterPartitionResponseData} -import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} -import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState} -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 scala.collection.{Map, Seq, Set, immutable, mutable} -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters._ -import scala.util.{Failure, Try} - -sealed trait ElectionTrigger -case object AutoTriggered extends ElectionTrigger -case object ZkTriggered extends ElectionTrigger -case object AdminClientTriggered extends ElectionTrigger - -object KafkaController extends Logging { - val InitialControllerEpoch = 0 - val InitialControllerEpochZkVersion = 0 - - type ElectLeadersCallback = Map[TopicPartition, Either[ApiError, Int]] => Unit - type ListReassignmentsCallback = Either[Map[TopicPartition, ReplicaAssignment], ApiError] => Unit - type AlterReassignmentsCallback = Either[Map[TopicPartition, ApiError], ApiError] => Unit - type UpdateFeaturesCallback = Either[ApiError, Map[String, ApiError]] => Unit - - private val ActiveControllerCountMetricName = "ActiveControllerCount" - private val OfflinePartitionsCountMetricName = "OfflinePartitionsCount" - private val PreferredReplicaImbalanceCountMetricName = "PreferredReplicaImbalanceCount" - private val ControllerStateMetricName = "ControllerState" - private val GlobalTopicCountMetricName = "GlobalTopicCount" - private val GlobalPartitionCountMetricName = "GlobalPartitionCount" - private val TopicsToDeleteCountMetricName = "TopicsToDeleteCount" - private val ReplicasToDeleteCountMetricName = "ReplicasToDeleteCount" - private val TopicsIneligibleToDeleteCountMetricName = "TopicsIneligibleToDeleteCount" - private val ReplicasIneligibleToDeleteCountMetricName = "ReplicasIneligibleToDeleteCount" - private val ActiveBrokerCountMetricName = "ActiveBrokerCount" - private val FencedBrokerCountMetricName = "FencedBrokerCount" - - // package private for testing - private[controller] val MetricNames = Set( - ActiveControllerCountMetricName, - OfflinePartitionsCountMetricName, - PreferredReplicaImbalanceCountMetricName, - ControllerStateMetricName, - GlobalTopicCountMetricName, - GlobalPartitionCountMetricName, - TopicsToDeleteCountMetricName, - ReplicasToDeleteCountMetricName, - TopicsIneligibleToDeleteCountMetricName, - ReplicasIneligibleToDeleteCountMetricName, - ActiveBrokerCountMetricName, - FencedBrokerCountMetricName - ) -} - -class KafkaController(val config: KafkaConfig, - zkClient: KafkaZkClient, - time: Time, - metrics: Metrics, - initialBrokerInfo: BrokerInfo, - initialBrokerEpoch: Long, - tokenManager: DelegationTokenManager, - brokerFeatures: BrokerFeatures, - featureCache: ZkFinalizedFeatureCache, - threadNamePrefix: Option[String] = None) - extends ControllerEventProcessor with Logging { - - private val metricsGroup = new KafkaMetricsGroup(this.getClass) - - this.logIdent = s"[Controller id=${config.brokerId}] " - - private val brokerInfo = initialBrokerInfo - @volatile private var _brokerEpoch = initialBrokerEpoch - - private val stateChangeLogger = new StateChangeLogger(config.brokerId, inControllerContext = true, None) - val controllerContext = new ControllerContext - var controllerChannelManager = new ControllerChannelManager( - () => controllerContext.epoch, - config, - time, - metrics, - stateChangeLogger, - threadNamePrefix - ) - - // have a separate scheduler for the controller to be able to start and stop independently of the kafka server - // visible for testing - private[controller] val kafkaScheduler = new KafkaScheduler(1) - - // visible for testing - private[controller] val eventManager = new ControllerEventManager(config.brokerId, this, time, - controllerContext.stats.rateAndTimeMetrics) - - private val brokerRequestBatch = new ControllerBrokerRequestBatch(config, controllerChannelManager, - eventManager, controllerContext, stateChangeLogger) - val replicaStateMachine: ReplicaStateMachine = new ZkReplicaStateMachine(config, stateChangeLogger, controllerContext, zkClient, - new ControllerBrokerRequestBatch(config, controllerChannelManager, eventManager, controllerContext, stateChangeLogger)) - val partitionStateMachine: PartitionStateMachine = new ZkPartitionStateMachine(config, stateChangeLogger, controllerContext, zkClient, - new ControllerBrokerRequestBatch(config, controllerChannelManager, eventManager, controllerContext, stateChangeLogger)) - private val topicDeletionManager = new TopicDeletionManager(config, controllerContext, replicaStateMachine, - partitionStateMachine, new ControllerDeletionClient(this, zkClient)) - - private val controllerChangeHandler = new ControllerChangeHandler(eventManager) - private val brokerChangeHandler = new BrokerChangeHandler(eventManager) - private val brokerModificationsHandlers: mutable.Map[Int, BrokerModificationsHandler] = mutable.Map.empty - private val topicChangeHandler = new TopicChangeHandler(eventManager) - private val topicDeletionHandler = new TopicDeletionHandler(eventManager) - private val partitionModificationsHandlers: mutable.Map[String, PartitionModificationsHandler] = mutable.Map.empty - private val partitionReassignmentHandler = new PartitionReassignmentHandler(eventManager) - private val preferredReplicaElectionHandler = new PreferredReplicaElectionHandler(eventManager) - private val isrChangeNotificationHandler = new IsrChangeNotificationHandler(eventManager) - private val logDirEventNotificationHandler = new LogDirEventNotificationHandler(eventManager) - - @volatile var activeControllerId = -1 - @volatile private var offlinePartitionCount = 0 - @volatile private var preferredReplicaImbalanceCount = 0 - @volatile private var globalTopicCount = 0 - @volatile private var globalPartitionCount = 0 - @volatile private var topicsToDeleteCount = 0 - @volatile private var replicasToDeleteCount = 0 - @volatile private var ineligibleTopicsToDeleteCount = 0 - @volatile private var ineligibleReplicasToDeleteCount = 0 - @volatile private var activeBrokerCount = 0 - - /* single-thread scheduler to clean expired tokens */ - private val tokenCleanScheduler = new KafkaScheduler(1, true, "delegation-token-cleaner") - - metricsGroup.newGauge(ActiveControllerCountMetricName, () => if (isActive) 1 else 0) - metricsGroup.newGauge(OfflinePartitionsCountMetricName, () => offlinePartitionCount) - metricsGroup.newGauge(PreferredReplicaImbalanceCountMetricName, () => preferredReplicaImbalanceCount) - metricsGroup.newGauge(ControllerStateMetricName, () => state.value) - metricsGroup.newGauge(GlobalTopicCountMetricName, () => globalTopicCount) - metricsGroup.newGauge(GlobalPartitionCountMetricName, () => globalPartitionCount) - metricsGroup.newGauge(TopicsToDeleteCountMetricName, () => topicsToDeleteCount) - metricsGroup.newGauge(ReplicasToDeleteCountMetricName, () => replicasToDeleteCount) - metricsGroup.newGauge(TopicsIneligibleToDeleteCountMetricName, () => ineligibleTopicsToDeleteCount) - metricsGroup.newGauge(ReplicasIneligibleToDeleteCountMetricName, () => ineligibleReplicasToDeleteCount) - metricsGroup.newGauge(ActiveBrokerCountMetricName, () => activeBrokerCount) - // FencedBrokerCount metric is always 0 in the ZK controller. - metricsGroup.newGauge(FencedBrokerCountMetricName, () => 0) - - /** - * Returns true if this broker is the current controller. - */ - def isActive: Boolean = activeControllerId == config.brokerId - - def brokerEpoch: Long = _brokerEpoch - - def epoch: Int = controllerContext.epoch - - /** - * Invoked when the controller module of a Kafka server is started up. This does not assume that the current broker - * is the controller. It merely registers the session expiration listener and starts the controller leader - * elector - */ - def startup(): Unit = { - zkClient.registerStateChangeHandler(new StateChangeHandler { - override val name: String = StateChangeHandlers.ControllerHandler - override def afterInitializingSession(): Unit = { - eventManager.put(RegisterBrokerAndReelect) - } - override def beforeInitializingSession(): Unit = { - val queuedEvent = eventManager.clearAndPut(Expire) - - // Block initialization of the new session until the expiration event is being handled, - // which ensures that all pending events have been processed before creating the new session - queuedEvent.awaitProcessing() - } - }) - eventManager.put(Startup) - eventManager.start() - } - - /** - * Invoked when the controller module of a Kafka server is shutting down. If the broker was the current controller, - * it shuts down the partition and replica state machines. If not, those are a no-op. In addition to that, it also - * shuts down the controller channel manager, if one exists (i.e. if it was the current controller) - */ - def shutdown(): Unit = { - try { - eventManager.close() - onControllerResignation() - } finally { - removeMetrics() - } - } - - /** - * On controlled shutdown, the controller first determines the partitions that the - * shutting down broker leads, and moves leadership of those partitions to another broker - * that is in that partition's ISR. - * - * @param id Id of the broker to shutdown. - * @param brokerEpoch The broker epoch in the controlled shutdown request - * @return The number of partitions that the broker still leads. - */ - def controlledShutdown(id: Int, brokerEpoch: Long, controlledShutdownCallback: Try[Set[TopicPartition]] => Unit): Unit = { - val controlledShutdownEvent = ControlledShutdown(id, brokerEpoch, controlledShutdownCallback) - eventManager.put(controlledShutdownEvent) - } - - private[kafka] def enableTopicUncleanLeaderElection(topic: String): Unit = { - if (isActive) { - eventManager.put(TopicUncleanLeaderElectionEnable(topic)) - } - } - - def isTopicQueuedForDeletion(topic: String): Boolean = { - topicDeletionManager.isTopicQueuedUpForDeletion(topic) - } - - private def state: ControllerState = eventManager.state - - /** - * This callback is invoked by the zookeeper leader elector on electing the current broker as the new controller. - * It does the following things on the become-controller state change - - * 1. Initializes the controller's context object that holds cache objects for current topics, live brokers and - * leaders for all existing partitions. - * 2. Starts the controller's channel manager - * 3. Starts the replica state machine - * 4. Starts the partition state machine - * If it encounters any unexpected exception/error while becoming controller, it resigns as the current controller. - * This ensures another controller election will be triggered and there will always be an actively serving controller - */ - private def onControllerFailover(): Unit = { - enableFeatureVersioning() - - info("Registering handlers") - - // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks - val childChangeHandlers = Seq(brokerChangeHandler, topicChangeHandler, topicDeletionHandler, logDirEventNotificationHandler, - isrChangeNotificationHandler) - childChangeHandlers.foreach(zkClient.registerZNodeChildChangeHandler) - - val nodeChangeHandlers = Seq(preferredReplicaElectionHandler, partitionReassignmentHandler) - nodeChangeHandlers.foreach(zkClient.registerZNodeChangeHandlerAndCheckExistence) - - info("Deleting log dir event notifications") - zkClient.deleteLogDirEventNotifications(controllerContext.epochZkVersion) - info("Deleting isr change notifications") - zkClient.deleteIsrChangeNotifications(controllerContext.epochZkVersion) - info("Initializing controller context") - initializeControllerContext() - info("Fetching topic deletions in progress") - val (topicsToBeDeleted, topicsIneligibleForDeletion) = fetchTopicDeletionsInProgress() - info("Initializing topic deletion manager") - topicDeletionManager.init(topicsToBeDeleted, topicsIneligibleForDeletion) - - // We need to send UpdateMetadataRequest after the controller context is initialized and before the state machines - // are started. The is because brokers need to receive the list of live brokers from UpdateMetadataRequest before - // they can process the LeaderAndIsrRequests that are generated by replicaStateMachine.startup() and - // partitionStateMachine.startup(). - info("Sending update metadata request") - sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set.empty) - - replicaStateMachine.startup() - partitionStateMachine.startup() - - info(s"Ready to serve as the new controller with epoch $epoch") - - initializePartitionReassignments() - topicDeletionManager.tryTopicDeletion() - val pendingPreferredReplicaElections = fetchPendingPreferredReplicaElections() - onReplicaElection(pendingPreferredReplicaElections, ElectionType.PREFERRED, ZkTriggered) - info("Starting the controller scheduler") - kafkaScheduler.startup() - if (config.autoLeaderRebalanceEnable) { - scheduleAutoLeaderRebalanceTask(delay = 5, unit = TimeUnit.SECONDS) - } - - if (config.tokenAuthEnabled) { - info("starting the token expiry check scheduler") - tokenCleanScheduler.startup() - tokenCleanScheduler.schedule("delete-expired-tokens", - () => tokenManager.expireTokens(), - 0L, - config.delegationTokenExpiryCheckIntervalMs) - } - } - - private def createFeatureZNode(newNode: FeatureZNode): Int = { - info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode") - zkClient.createFeatureZNode(newNode) - val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path) - newVersion - } - - private def updateFeatureZNode(updatedNode: FeatureZNode): Int = { - info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode") - zkClient.updateFeatureZNode(updatedNode) - } - - /** - * This method enables the feature versioning system (KIP-584). - * - * Development in Kafka (from a high level) is organized into features. Each feature is tracked by - * a name and a range of version numbers or a version number. A feature can be of two types: - * - * 1. Supported feature: - * A supported feature is represented by a name (string) and a range of versions (defined by a - * SupportedVersionRange). It refers to a feature that a particular broker advertises support for. - * Each broker advertises the version ranges of its own supported features in its own - * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and - * do not represent any guarantee of a cluster-wide availability of the feature for any particular - * range of versions. - * - * 2. Finalized feature: - * A finalized feature is represented by a name (string) and a specified version level (defined - * by a Short). Whenever the feature versioning system (KIP-584) is - * enabled, the finalized features are stored in the cluster-wide common FeatureZNode. - * In comparison to a supported feature, the key difference is that a finalized feature exists - * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a - * specified range of version levels. Also, the controller is the only entity modifying the - * information about finalized features. - * - * This method sets up the FeatureZNode with enabled status, which means that the finalized - * features stored in the FeatureZNode are active. The enabled status should be written by the - * controller to the FeatureZNode only when the broker IBP config is greater than or equal to - * IBP_2_7_IV0. - * - * There are multiple cases handled here: - * - * 1. New cluster bootstrap: - * A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config - * setting greater than or equal to IBP_2_7_IV0. We would like to start the cluster with all - * the possible supported features finalized immediately. Assuming this is the case, the - * controller will start up and notice that the FeatureZNode is absent in the new cluster, - * it will then create a FeatureZNode (with enabled status) containing the entire list of - * supported features as its finalized features. - * - * 2. Broker binary upgraded, but IBP config set to lower than IBP_2_7_IV0: - * Imagine there was an existing Kafka cluster with IBP config less than IBP_2_7_IV0, and the - * broker binary has now been upgraded to a newer version that supports the feature versioning - * system (KIP-584). But the IBP config is still set to lower than IBP_2_7_IV0, and may be - * set to a higher value later. In this case, we want to start with no finalized features and - * allow the user to finalize them whenever they are ready i.e. in the future whenever the - * user sets IBP config to be greater than or equal to IBP_2_7_IV0, then the user could start - * finalizing the features. This process ensures we do not enable all the possible features - * immediately after an upgrade, which could be harmful to Kafka. - * This is how we handle such a case: - * - Before the IBP config upgrade (i.e. IBP config set to less than IBP_2_7_IV0), the - * controller will start up and check if the FeatureZNode is absent. - * - If the node is absent, it will react by creating a FeatureZNode with disabled status - * and empty finalized features. - * - Otherwise, if a node already exists in enabled status then the controller will just - * flip the status to disabled and clear the finalized features. - * - After the IBP config upgrade (i.e. IBP config set to greater than or equal to - * IBP_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists - * and whether it is disabled. - * - If the node is in disabled status, the controller won’t upgrade all features immediately. - * Instead it will just switch the FeatureZNode status to enabled status. This lets the - * user finalize the features later. - * - Otherwise, if a node already exists in enabled status then the controller will leave - * the node umodified. - * - * 3. Broker binary upgraded, with existing cluster IBP config >= IBP_2_7_IV0: - * Imagine there was an existing Kafka cluster with IBP config >= IBP_2_7_IV0, and the broker - * binary has just been upgraded to a newer version (that supports IBP config IBP_2_7_IV0 and - * higher). The controller will start up and find that a FeatureZNode is already present with - * enabled status and existing finalized features. In such a case, the controller leaves the node - * unmodified. - * - * 4. Broker downgrade: - * Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to - * IBP_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a - * value less than IBP_2_7_IV0. This means the user is also disabling the feature versioning - * system (KIP-584). In this case, when the controller starts up with the lower IBP config, it - * will switch the FeatureZNode status to disabled with empty features. - */ - private def enableFeatureVersioning(): Unit = { - val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) - if (version == ZkVersion.UnknownVersion) { - val newVersion = createFeatureZNode( - FeatureZNode(config.interBrokerProtocolVersion, - FeatureZNodeStatus.Enabled, - brokerFeatures.defaultFinalizedFeatures.asScala.map { case (k, v) => (k, v.shortValue()) } - )) - featureCache.waitUntilFeatureEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) - } else { - val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) - val newFeatures = existingFeatureZNode.status match { - case FeatureZNodeStatus.Enabled => existingFeatureZNode.features - case FeatureZNodeStatus.Disabled => - if (existingFeatureZNode.features.nonEmpty) { - warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" + - s" contains non-empty features: ${existingFeatureZNode.features}") - } - Map.empty[String, Short] - } - val newFeatureZNode = FeatureZNode(config.interBrokerProtocolVersion, FeatureZNodeStatus.Enabled, newFeatures) - if (!newFeatureZNode.equals(existingFeatureZNode)) { - val newVersion = updateFeatureZNode(newFeatureZNode) - featureCache.waitUntilFeatureEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) - } - } - } - - private def scheduleAutoLeaderRebalanceTask(delay: Long, unit: TimeUnit): Unit = { - kafkaScheduler.scheduleOnce("auto-leader-rebalance-task", - () => eventManager.put(AutoPreferredReplicaLeaderElection), - unit.toMillis(delay)) - } - - /** - * This callback is invoked by the zookeeper leader elector when the current broker resigns as the controller. This is - * required to clean up internal controller data structures - */ - private def onControllerResignation(): Unit = { - debug("Resigning") - // de-register listeners - zkClient.unregisterZNodeChildChangeHandler(isrChangeNotificationHandler.path) - zkClient.unregisterZNodeChangeHandler(partitionReassignmentHandler.path) - zkClient.unregisterZNodeChangeHandler(preferredReplicaElectionHandler.path) - zkClient.unregisterZNodeChildChangeHandler(logDirEventNotificationHandler.path) - unregisterBrokerModificationsHandler(brokerModificationsHandlers.keySet) - - // shutdown leader rebalance scheduler - kafkaScheduler.shutdown() - - // stop token expiry check scheduler - tokenCleanScheduler.shutdown() - - // shutdown partition state machine - partitionStateMachine.shutdown() - zkClient.unregisterZNodeChildChangeHandler(topicChangeHandler.path) - unregisterPartitionModificationsHandlers(partitionModificationsHandlers.keys.toSeq) - zkClient.unregisterZNodeChildChangeHandler(topicDeletionHandler.path) - // shutdown replica state machine - replicaStateMachine.shutdown() - zkClient.unregisterZNodeChildChangeHandler(brokerChangeHandler.path) - - controllerChannelManager.shutdown() - controllerContext.resetContext() - - info("Resigned") - } - - private def removeMetrics(): Unit = { - KafkaController.MetricNames.foreach(metricsGroup.removeMetric) - } - - /* - * This callback is invoked by the controller's LogDirEventNotificationListener with the list of broker ids who - * have experienced new log directory failures. In response the controller should send LeaderAndIsrRequest - * to all these brokers to query the state of their replicas. Replicas with an offline log directory respond with - * KAFKA_STORAGE_ERROR, which will be handled by the LeaderAndIsrResponseReceived event. - */ - private def onBrokerLogDirFailure(brokerIds: Seq[Int]): Unit = { - // send LeaderAndIsrRequest for all replicas on those brokers to see if they are still online. - info(s"Handling log directory failure for brokers ${brokerIds.mkString(",")}") - val replicasOnBrokers = controllerContext.replicasOnBrokers(brokerIds.toSet) - replicaStateMachine.handleStateChanges(replicasOnBrokers.toSeq, OnlineReplica) - } - - /** - * This callback is invoked by the replica state machine's broker change listener, with the list of newly started - * brokers as input. It does the following - - * 1. Sends update metadata request to all live and shutting down brokers - * 2. Triggers the OnlinePartition state change for all new/offline partitions - * 3. It checks whether there are reassigned replicas assigned to any newly started brokers. If - * so, it performs the reassignment logic for each topic/partition. - * - * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point for two reasons: - * 1. The partition state machine, when triggering online state change, will refresh leader and ISR for only those - * partitions currently new or offline (rather than every partition this controller is aware of) - * 2. Even if we do refresh the cache, there is no guarantee that by the time the leader and ISR request reaches - * every broker that it is still valid. Brokers check the leader epoch to determine validity of the request. - */ - private def onBrokerStartup(newBrokers: Seq[Int]): Unit = { - info(s"New broker startup callback for ${newBrokers.mkString(",")}") - newBrokers.foreach(controllerContext.replicasOnOfflineDirs.remove) - val newBrokersSet = newBrokers.toSet - val existingBrokers = controllerContext.liveOrShuttingDownBrokerIds.diff(newBrokersSet) - // Send update metadata request to all the existing brokers in the cluster so that they know about the new brokers - // via this update. No need to include any partition states in the request since there are no partition state changes. - sendUpdateMetadataRequest(existingBrokers.toSeq, Set.empty) - // Send update metadata request to all the new brokers in the cluster with a full set of partition states for initialization. - // In cases of controlled shutdown leaders will not be elected when a new broker comes up. So at least in the - // common controlled shutdown case, the metadata will reach the new brokers faster. - sendUpdateMetadataRequest(newBrokers, controllerContext.partitionsWithLeaders) - // the very first thing to do when a new broker comes up is send it the entire list of partitions that it is - // supposed to host. Based on that the broker starts the high watermark threads for the input list of partitions - val allReplicasOnNewBrokers = controllerContext.replicasOnBrokers(newBrokersSet) - replicaStateMachine.handleStateChanges(allReplicasOnNewBrokers.toSeq, OnlineReplica) - // when a new broker comes up, the controller needs to trigger leader election for all new and offline partitions - // to see if these brokers can become leaders for some/all of those - partitionStateMachine.triggerOnlinePartitionStateChange() - // check if reassignment of some partitions need to be restarted - maybeResumeReassignments { (_, assignment) => - assignment.targetReplicas.exists(newBrokersSet.contains) - } - // check if topic deletion needs to be resumed. If at least one replica that belongs to the topic being deleted exists - // on the newly restarted brokers, there is a chance that topic deletion can resume - val replicasForTopicsToBeDeleted = allReplicasOnNewBrokers.filter(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) - if (replicasForTopicsToBeDeleted.nonEmpty) { - info(s"Some replicas ${replicasForTopicsToBeDeleted.mkString(",")} for topics scheduled for deletion " + - s"${controllerContext.topicsToBeDeleted.mkString(",")} are on the newly restarted brokers " + - s"${newBrokers.mkString(",")}. Signaling restart of topic deletion for these topics") - topicDeletionManager.resumeDeletionForTopics(replicasForTopicsToBeDeleted.map(_.topic)) - } - registerBrokerModificationsHandler(newBrokers) - } - - private def maybeResumeReassignments(shouldResume: (TopicPartition, ReplicaAssignment) => Boolean): Unit = { - controllerContext.partitionsBeingReassigned.foreach { tp => - val currentAssignment = controllerContext.partitionFullReplicaAssignment(tp) - if (shouldResume(tp, currentAssignment)) - onPartitionReassignment(tp, currentAssignment) - } - } - - private def registerBrokerModificationsHandler(brokerIds: Iterable[Int]): Unit = { - debug(s"Register BrokerModifications handler for $brokerIds") - brokerIds.foreach { brokerId => - val brokerModificationsHandler = new BrokerModificationsHandler(eventManager, brokerId) - zkClient.registerZNodeChangeHandlerAndCheckExistence(brokerModificationsHandler) - brokerModificationsHandlers.put(brokerId, brokerModificationsHandler) - } - } - - private def unregisterBrokerModificationsHandler(brokerIds: Iterable[Int]): Unit = { - debug(s"Unregister BrokerModifications handler for $brokerIds") - brokerIds.foreach { brokerId => - brokerModificationsHandlers.remove(brokerId).foreach(handler => zkClient.unregisterZNodeChangeHandler(handler.path)) - } - } - - /* - * This callback is invoked by the replica state machine's broker change listener with the list of failed brokers - * as input. It will call onReplicaBecomeOffline(...) with the list of replicas on those failed brokers as input. - */ - private def onBrokerFailure(deadBrokers: Seq[Int]): Unit = { - info(s"Broker failure callback for ${deadBrokers.mkString(",")}") - deadBrokers.foreach(controllerContext.replicasOnOfflineDirs.remove) - val deadBrokersThatWereShuttingDown = - deadBrokers.filter(id => controllerContext.shuttingDownBrokerIds.remove(id)) - if (deadBrokersThatWereShuttingDown.nonEmpty) - info(s"Removed ${deadBrokersThatWereShuttingDown.mkString(",")} from list of shutting down brokers.") - val allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokers.toSet) - onReplicasBecomeOffline(allReplicasOnDeadBrokers) - - unregisterBrokerModificationsHandler(deadBrokers) - } - - private def onBrokerUpdate(updatedBrokerId: Int): Unit = { - info(s"Broker info update callback for $updatedBrokerId") - sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set.empty) - } - - /** - * This method marks the given replicas as offline. It does the following - - * 1. Marks the given partitions as offline - * 2. Triggers the OnlinePartition state change for all new/offline partitions - * 3. Invokes the OfflineReplica state change on the input list of newly offline replicas - * 4. If no partitions are affected then send UpdateMetadataRequest to live or shutting down brokers - * - * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because - * the partition state machine will refresh our cache for us when performing leader election for all new/offline - * partitions coming online. - */ - private def onReplicasBecomeOffline(newOfflineReplicas: Set[PartitionAndReplica]): Unit = { - val (newOfflineReplicasForDeletion, newOfflineReplicasNotForDeletion) = - newOfflineReplicas.partition(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) - - val partitionsWithOfflineLeader = controllerContext.partitionsWithOfflineLeader - - // trigger OfflinePartition state for all partitions whose current leader is one amongst the newOfflineReplicas - partitionStateMachine.handleStateChanges(partitionsWithOfflineLeader.toSeq, OfflinePartition) - // trigger OnlinePartition state changes for offline or new partitions - val onlineStateChangeResults = partitionStateMachine.triggerOnlinePartitionStateChange() - // trigger OfflineReplica state change for those newly offline replicas - replicaStateMachine.handleStateChanges(newOfflineReplicasNotForDeletion.toSeq, OfflineReplica) - - // fail deletion of topics that are affected by the offline replicas - if (newOfflineReplicasForDeletion.nonEmpty) { - // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be - // deleted when its log directory is offline. This will prevent the replica from being in TopicDeletionStarted state indefinitely - // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state - topicDeletionManager.failReplicaDeletion(newOfflineReplicasForDeletion) - } - - // If no partition has changed leader or ISR, no UpdateMetadataRequest is sent through PartitionStateMachine - // and ReplicaStateMachine. In that case, we want to send an UpdateMetadataRequest explicitly to - // propagate the information about the new offline brokers. - if (newOfflineReplicasNotForDeletion.isEmpty && onlineStateChangeResults.values.forall(_.isLeft)) { - sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set.empty) - } - } - - /** - * This callback is invoked by the topic change callback with the list of failed brokers as input. - * It does the following - - * 1. Move the newly created partitions to the NewPartition state - * 2. Move the newly created partitions from NewPartition->OnlinePartition state - */ - private def onNewPartitionCreation(newPartitions: Set[TopicPartition]): Unit = { - info(s"New partition creation callback for ${newPartitions.mkString(",")}") - partitionStateMachine.handleStateChanges(newPartitions.toSeq, NewPartition) - replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions).toSeq, NewReplica) - partitionStateMachine.handleStateChanges( - newPartitions.toSeq, - OnlinePartition, - Some(OfflinePartitionLeaderElectionStrategy(false)) - ) - replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions).toSeq, OnlineReplica) - } - - /** - * This callback is invoked: - * 1. By the AlterPartitionReassignments API - * 2. By the reassigned partitions listener which is triggered when the /admin/reassign/partitions znode is created - * 3. When an ongoing reassignment finishes - this is detected by a change in the partition's ISR znode - * 4. Whenever a new broker comes up which is part of an ongoing reassignment - * 5. On controller startup/failover - * - * Reassigning replicas for a partition goes through a few steps listed in the code. - * RS = current assigned replica set - * ORS = Original replica set for partition - * TRS = Reassigned (target) replica set - * AR = The replicas we are adding as part of this reassignment - * RR = The replicas we are removing as part of this reassignment - * - * A reassignment may have up to three phases, each with its own steps: - - * Phase U (Assignment update): Regardless of the trigger, the first step is in the reassignment process - * is to update the existing assignment state. We always update the state in Zookeeper before - * we update memory so that it can be resumed upon controller fail-over. - * - * U1. Update ZK with RS = ORS + TRS, AR = TRS - ORS, RR = ORS - TRS. - * U2. Update memory with RS = ORS + TRS, AR = TRS - ORS and RR = ORS - TRS - * U3. If we are cancelling or replacing an existing reassignment, send StopReplica to all members - * of AR in the original reassignment if they are not in TRS from the new assignment - * - * To complete the reassignment, we need to bring the new replicas into sync, so depending on the state - * of the ISR, we will execute one of the following steps. - * - * Phase A (when TRS != ISR): The reassignment is not yet complete - * - * A1. Bump the leader epoch for the partition and send LeaderAndIsr updates to RS. - * A2. Start new replicas AR by moving replicas in AR to NewReplica state. - * - * Phase B (when TRS = ISR): The reassignment is complete - * - * B1. Move all replicas in AR to OnlineReplica state. - * B2. Set RS = TRS, AR = [], RR = [] in memory. - * B3. Send a LeaderAndIsr request with RS = TRS. This will prevent the leader from adding any replica in TRS - ORS back in the isr. - * If the current leader is not in TRS or isn't alive, we move the leader to a new replica in TRS. - * We may send the LeaderAndIsr to more than the TRS replicas due to the - * way the partition state machine works (it reads replicas from ZK) - * B4. Move all replicas in RR to OfflineReplica state. As part of OfflineReplica state change, we shrink the - * isr to remove RR in ZooKeeper and send a LeaderAndIsr ONLY to the Leader to notify it of the shrunk isr. - * After that, we send a StopReplica (delete = false) to the replicas in RR. - * B5. Move all replicas in RR to NonExistentReplica state. This will send a StopReplica (delete = true) to - * the replicas in RR to physically delete the replicas on disk. - * B6. Update ZK with RS=TRS, AR=[], RR=[]. - * B7. Remove the ISR reassign listener and maybe update the /admin/reassign_partitions path in ZK to remove this partition from it if present. - * B8. After electing leader, the replicas and isr information changes. So resend the update metadata request to every broker. - * - * In general, there are two goals we want to aim for: - * 1. Every replica present in the replica set of a LeaderAndIsrRequest gets the request sent to it - * 2. Replicas that are removed from a partition's assignment get StopReplica sent to them - * - * For example, if ORS = {1,2,3} and TRS = {4,5,6}, the values in the topic and leader/isr paths in ZK - * may go through the following transitions. - * RS AR RR leader isr - * {1,2,3} {} {} 1 {1,2,3} (initial state) - * {4,5,6,1,2,3} {4,5,6} {1,2,3} 1 {1,2,3} (step A2) - * {4,5,6,1,2,3} {4,5,6} {1,2,3} 1 {1,2,3,4,5,6} (phase B) - * {4,5,6,1,2,3} {4,5,6} {1,2,3} 4 {1,2,3,4,5,6} (step B3) - * {4,5,6,1,2,3} {4,5,6} {1,2,3} 4 {4,5,6} (step B4) - * {4,5,6} {} {} 4 {4,5,6} (step B6) - * - * Note that we have to update RS in ZK with TRS last since it's the only place where we store ORS persistently. - * This way, if the controller crashes before that step, we can still recover. - */ - private def onPartitionReassignment(topicPartition: TopicPartition, reassignment: ReplicaAssignment): Unit = { - // While a reassignment is in progress, deletion is not allowed - topicDeletionManager.markTopicIneligibleForDeletion(Set(topicPartition.topic), reason = "topic reassignment in progress") - - updateCurrentReassignment(topicPartition, reassignment) - - val addingReplicas = reassignment.addingReplicas - val removingReplicas = reassignment.removingReplicas - - if (!isReassignmentComplete(topicPartition, reassignment)) { - // A1. Send LeaderAndIsr request to every replica in ORS + TRS (with the new RS, AR and RR). - updateLeaderEpochAndSendRequest(topicPartition, reassignment) - // A2. replicas in AR -> NewReplica - startNewReplicasForReassignedPartition(topicPartition, addingReplicas) - } else { - // B1. replicas in AR -> OnlineReplica - replicaStateMachine.handleStateChanges(addingReplicas.map(PartitionAndReplica(topicPartition, _)), OnlineReplica) - // B2. Set RS = TRS, AR = [], RR = [] in memory. - val completedReassignment = ReplicaAssignment(reassignment.targetReplicas) - controllerContext.updatePartitionFullReplicaAssignment(topicPartition, completedReassignment) - // B3. Send LeaderAndIsr request with a potential new leader (if current leader not in TRS) and - // a new RS (using TRS) and same isr to every broker in ORS + TRS or TRS - moveReassignedPartitionLeaderIfRequired(topicPartition, completedReassignment) - // B4. replicas in RR -> Offline (force those replicas out of isr) - // B5. replicas in RR -> NonExistentReplica (force those replicas to be deleted) - stopRemovedReplicasOfReassignedPartition(topicPartition, removingReplicas) - // B6. Update ZK with RS = TRS, AR = [], RR = []. - updateReplicaAssignmentForPartition(topicPartition, completedReassignment) - // B7. Remove the ISR reassign listener and maybe update the /admin/reassign_partitions path in ZK to remove this partition from it. - removePartitionFromReassigningPartitions(topicPartition, completedReassignment) - // B8. After electing a leader in B3, the replicas and isr information changes, so resend the update metadata request to every broker - sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition)) - // signal delete topic thread if reassignment for some partitions belonging to topics being deleted just completed - topicDeletionManager.resumeDeletionForTopics(Set(topicPartition.topic)) - } - } - - /** - * Update the current assignment state in Zookeeper and in memory. If a reassignment is already in - * progress, then the new reassignment will supplant it and some replicas will be shutdown. - * - * Note that due to the way we compute the original replica set, we cannot guarantee that a - * cancellation will restore the original replica order. Target replicas are always listed - * first in the replica set in the desired order, which means we have no way to get to the - * original order if the reassignment overlaps with the current assignment. For example, - * with an initial assignment of [1, 2, 3] and a reassignment of [3, 4, 2], then the replicas - * will be encoded as [3, 4, 2, 1] while the reassignment is in progress. If the reassignment - * is cancelled, there is no way to restore the original order. - * - * @param topicPartition The reassigning partition - * @param reassignment The new reassignment - */ - private def updateCurrentReassignment(topicPartition: TopicPartition, reassignment: ReplicaAssignment): Unit = { - val currentAssignment = controllerContext.partitionFullReplicaAssignment(topicPartition) - - if (currentAssignment != reassignment) { - debug(s"Updating assignment of partition $topicPartition from $currentAssignment to $reassignment") - - // U1. Update assignment state in zookeeper - updateReplicaAssignmentForPartition(topicPartition, reassignment) - // U2. Update assignment state in memory - controllerContext.updatePartitionFullReplicaAssignment(topicPartition, reassignment) - - // If there is a reassignment already in progress, then some of the currently adding replicas - // may be eligible for immediate removal, in which case we need to stop the replicas. - val unneededReplicas = currentAssignment.replicas.diff(reassignment.replicas) - if (unneededReplicas.nonEmpty) - stopRemovedReplicasOfReassignedPartition(topicPartition, unneededReplicas) - } - - controllerContext.partitionsBeingReassigned.add(topicPartition) - } - - /** - * Trigger a partition reassignment provided that the topic exists and is not being deleted. - * - * This is called when a reassignment is initially received either through Zookeeper or through the - * AlterPartitionReassignments API - * - * The `partitionsBeingReassigned` field in the controller context will be updated by this - * call after the reassignment completes validation and is successfully stored in the topic - * assignment zNode. - * - * @param reassignments The reassignments to begin processing - * @return A map of any errors in the reassignment. If the error is NONE for a given partition, - * then the reassignment was submitted successfully. - */ - private def maybeTriggerPartitionReassignment(reassignments: Map[TopicPartition, ReplicaAssignment]): Map[TopicPartition, ApiError] = { - reassignments.map { case (tp, reassignment) => - val topic = tp.topic - - val apiError = if (topicDeletionManager.isTopicQueuedUpForDeletion(topic)) { - info(s"Skipping reassignment of $tp since the topic is currently being deleted") - new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, "The partition does not exist.") - } else { - val assignedReplicas = controllerContext.partitionReplicaAssignment(tp) - if (assignedReplicas.nonEmpty) { - try { - onPartitionReassignment(tp, reassignment) - ApiError.NONE - } catch { - case e: ControllerMovedException => - info(s"Failed completing reassignment of partition $tp because controller has moved to another broker") - throw e - case e: Throwable => - error(s"Error completing reassignment of partition $tp", e) - new ApiError(Errors.UNKNOWN_SERVER_ERROR) - } - } else { - new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, "The partition does not exist.") - } - } - - tp -> apiError - } - } - - /** - * Attempt to elect a replica as leader for each of the given partitions. - * @param partitions The partitions to have a new leader elected - * @param electionType The type of election to perform - * @param electionTrigger The reason for trigger this election - * @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[this] def onReplicaElection( - partitions: Set[TopicPartition], - electionType: ElectionType, - electionTrigger: ElectionTrigger - ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - info(s"Starting replica leader election ($electionType) for partitions ${partitions.mkString(",")} triggered by $electionTrigger") - try { - val strategy = electionType match { - case ElectionType.PREFERRED => PreferredReplicaPartitionLeaderElectionStrategy - case ElectionType.UNCLEAN => - /* Let's be conservative and only trigger unclean election if the election type is unclean and it was - * triggered by the admin client - */ - OfflinePartitionLeaderElectionStrategy(allowUnclean = electionTrigger == AdminClientTriggered) - } - - val results = partitionStateMachine.handleStateChanges( - partitions.toSeq, - OnlinePartition, - Some(strategy) - ) - if (electionTrigger != AdminClientTriggered) { - results.foreach { - case (tp, Left(throwable)) => - if (throwable.isInstanceOf[ControllerMovedException]) { - info(s"Error completing replica leader election ($electionType) for partition $tp because controller has moved to another broker.", throwable) - throw throwable - } else { - error(s"Error completing replica leader election ($electionType) for partition $tp", throwable) - } - case (_, Right(_)) => // Ignored; No need to log or throw exception for the success cases - } - } - - results - } finally { - if (electionTrigger != AdminClientTriggered) { - removePartitionsFromPreferredReplicaElection(partitions, electionTrigger == AutoTriggered) - } - } - } - - private def initializeControllerContext(): Unit = { - // update controller cache with delete topic information - val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster - val (compatibleBrokerAndEpochs, incompatibleBrokerAndEpochs) = partitionOnFeatureCompatibility(curBrokerAndEpochs) - if (incompatibleBrokerAndEpochs.nonEmpty) { - warn("Ignoring registration of new brokers due to incompatibilities with finalized features: " + - incompatibleBrokerAndEpochs.map { case (broker, _) => broker.id }.toSeq.sorted.mkString(",")) - } - controllerContext.setLiveBrokers(compatibleBrokerAndEpochs) - info(s"Initialized broker epochs cache: ${controllerContext.liveBrokerIdAndEpochs}") - controllerContext.setAllTopics(zkClient.getAllTopicsInCluster(true)) - registerPartitionModificationsHandlers(controllerContext.allTopics.toSeq) - val replicaAssignmentAndTopicIds = zkClient.getReplicaAssignmentAndTopicIdForTopics(controllerContext.allTopics.toSet) - processTopicIds(replicaAssignmentAndTopicIds) - - replicaAssignmentAndTopicIds.foreach { case TopicIdReplicaAssignment(_, _, assignments) => - assignments.foreach { case (topicPartition, replicaAssignment) => - controllerContext.updatePartitionFullReplicaAssignment(topicPartition, replicaAssignment) - if (replicaAssignment.isBeingReassigned) - controllerContext.partitionsBeingReassigned.add(topicPartition) - } - } - controllerContext.clearPartitionLeadershipInfo() - controllerContext.shuttingDownBrokerIds.clear() - // register broker modifications handlers - registerBrokerModificationsHandler(controllerContext.liveOrShuttingDownBrokerIds) - // update the leader and isr cache for all existing partitions from Zookeeper - updateLeaderAndIsrCache() - // start the channel manager - controllerChannelManager.startup(controllerContext.liveOrShuttingDownBrokers) - info(s"Currently active brokers in the cluster: ${controllerContext.liveBrokerIds}") - info(s"Currently shutting brokers in the cluster: ${controllerContext.shuttingDownBrokerIds}") - info(s"Current list of topics in the cluster: ${controllerContext.allTopics}") - } - - private def fetchPendingPreferredReplicaElections(): Set[TopicPartition] = { - val partitionsUndergoingPreferredReplicaElection = zkClient.getPreferredReplicaElection - // check if they are already completed or topic was deleted - val partitionsThatCompletedPreferredReplicaElection = partitionsUndergoingPreferredReplicaElection.filter { partition => - val replicas = controllerContext.partitionReplicaAssignment(partition) - val topicDeleted = replicas.isEmpty - val successful = - if (!topicDeleted) controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader == replicas.head else false - successful || topicDeleted - } - val pendingPreferredReplicaElectionsIgnoringTopicDeletion = partitionsUndergoingPreferredReplicaElection -- partitionsThatCompletedPreferredReplicaElection - val pendingPreferredReplicaElectionsSkippedFromTopicDeletion = pendingPreferredReplicaElectionsIgnoringTopicDeletion.filter(partition => topicDeletionManager.isTopicQueuedUpForDeletion(partition.topic)) - val pendingPreferredReplicaElections = pendingPreferredReplicaElectionsIgnoringTopicDeletion -- pendingPreferredReplicaElectionsSkippedFromTopicDeletion - info(s"Partitions undergoing preferred replica election: ${partitionsUndergoingPreferredReplicaElection.mkString(",")}") - info(s"Partitions that completed preferred replica election: ${partitionsThatCompletedPreferredReplicaElection.mkString(",")}") - info(s"Skipping preferred replica election for partitions due to topic deletion: ${pendingPreferredReplicaElectionsSkippedFromTopicDeletion.mkString(",")}") - info(s"Resuming preferred replica election for partitions: ${pendingPreferredReplicaElections.mkString(",")}") - pendingPreferredReplicaElections - } - - /** - * Initialize pending reassignments. This includes reassignments sent through /admin/reassign_partitions, - * which will supplant any API reassignments already in progress. - */ - private def initializePartitionReassignments(): Unit = { - // New reassignments may have been submitted through Zookeeper while the controller was failing over - val zkPartitionsResumed = processZkPartitionReassignment() - // We may also have some API-based reassignments that need to be restarted - maybeResumeReassignments { (tp, _) => - !zkPartitionsResumed.contains(tp) - } - } - - private def fetchTopicDeletionsInProgress(): (Set[String], Set[String]) = { - val topicsToBeDeleted = zkClient.getTopicDeletions.toSet - val topicsWithOfflineReplicas = controllerContext.allTopics.filter { topic => { - val replicasForTopic = controllerContext.replicasForTopic(topic) - replicasForTopic.exists(r => !controllerContext.isReplicaOnline(r.replica, r.topicPartition)) - }} - val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsBeingReassigned.map(_.topic) - val topicsIneligibleForDeletion = topicsWithOfflineReplicas | topicsForWhichPartitionReassignmentIsInProgress - info(s"List of topics to be deleted: ${topicsToBeDeleted.mkString(",")}") - info(s"List of topics ineligible for deletion: ${topicsIneligibleForDeletion.mkString(",")}") - (topicsToBeDeleted, topicsIneligibleForDeletion) - } - - private def updateLeaderAndIsrCache(partitions: Seq[TopicPartition] = controllerContext.allPartitions.toSeq): Unit = { - val leaderIsrAndControllerEpochs = zkClient.getTopicPartitionStates(partitions) - leaderIsrAndControllerEpochs.foreachEntry { (partition, leaderIsrAndControllerEpoch) => - controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) - } - } - - private def isReassignmentComplete(partition: TopicPartition, assignment: ReplicaAssignment): Boolean = { - if (!assignment.isBeingReassigned) { - true - } else { - zkClient.getTopicPartitionStates(Seq(partition)).get(partition).exists { leaderIsrAndControllerEpoch => - val isr = leaderIsrAndControllerEpoch.leaderAndIsr.isr.asScala.toSet.map(Int.unbox) - val targetReplicas = assignment.targetReplicas.toSet - targetReplicas.subsetOf(isr) - } - } - } - - private def moveReassignedPartitionLeaderIfRequired(topicPartition: TopicPartition, - newAssignment: ReplicaAssignment): Unit = { - val reassignedReplicas = newAssignment.replicas - val currentLeader = controllerContext.partitionLeadershipInfo(topicPartition).get.leaderAndIsr.leader - - if (!reassignedReplicas.contains(currentLeader)) { - info(s"Leader $currentLeader for partition $topicPartition being reassigned, " + - s"is not in the new list of replicas ${reassignedReplicas.mkString(",")}. Re-electing leader") - // move the leader to one of the alive and caught up new replicas - partitionStateMachine.handleStateChanges(Seq(topicPartition), OnlinePartition, Some(ReassignPartitionLeaderElectionStrategy)) - } else if (controllerContext.isReplicaOnline(currentLeader, topicPartition)) { - info(s"Leader $currentLeader for partition $topicPartition being reassigned, " + - s"is already in the new list of replicas ${reassignedReplicas.mkString(",")} and is alive") - // shrink replication factor and update the leader epoch in zookeeper to use on the next LeaderAndIsrRequest - updateLeaderEpochAndSendRequest(topicPartition, newAssignment) - } else { - info(s"Leader $currentLeader for partition $topicPartition being reassigned, " + - s"is already in the new list of replicas ${reassignedReplicas.mkString(",")} but is dead") - partitionStateMachine.handleStateChanges(Seq(topicPartition), OnlinePartition, Some(ReassignPartitionLeaderElectionStrategy)) - } - } - - private def stopRemovedReplicasOfReassignedPartition(topicPartition: TopicPartition, - removedReplicas: Seq[Int]): Unit = { - // first move the replica to offline state (the controller removes it from the ISR) - val replicasToBeDeleted = removedReplicas.map(PartitionAndReplica(topicPartition, _)) - replicaStateMachine.handleStateChanges(replicasToBeDeleted, OfflineReplica) - // send stop replica command to the old replicas - replicaStateMachine.handleStateChanges(replicasToBeDeleted, ReplicaDeletionStarted) - // TODO: Eventually partition reassignment could use a callback that does retries if deletion failed - replicaStateMachine.handleStateChanges(replicasToBeDeleted, ReplicaDeletionSuccessful) - replicaStateMachine.handleStateChanges(replicasToBeDeleted, NonExistentReplica) - } - - private def updateReplicaAssignmentForPartition(topicPartition: TopicPartition, assignment: ReplicaAssignment): Unit = { - throw new UnsupportedOperationException() - } - - private def startNewReplicasForReassignedPartition(topicPartition: TopicPartition, newReplicas: Seq[Int]): Unit = { - // send the start replica request to the brokers in the reassigned replicas list that are not in the assigned - // replicas list - newReplicas.foreach { replica => - replicaStateMachine.handleStateChanges(Seq(PartitionAndReplica(topicPartition, replica)), NewReplica) - } - } - - private def updateLeaderEpochAndSendRequest(topicPartition: TopicPartition, - assignment: ReplicaAssignment): Unit = { - val stateChangeLog = stateChangeLogger.withControllerEpoch(controllerContext.epoch) - updateLeaderEpoch(topicPartition) match { - case Some(updatedLeaderIsrAndControllerEpoch) => - try { - brokerRequestBatch.newBatch() - // the isNew flag, when set to true, makes sure that when a replica possibly resided - // in a logDir that is offline, we refrain from just creating a new replica in a good - // logDir. This is exactly the behavior we want for the original replicas, but not - // for the replicas we add in this reassignment. For new replicas, want to be able - // to assign to one of the good logDirs. - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(assignment.originReplicas, topicPartition, - updatedLeaderIsrAndControllerEpoch, assignment, isNew = false) - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(assignment.addingReplicas, topicPartition, - updatedLeaderIsrAndControllerEpoch, assignment, isNew = true) - brokerRequestBatch.sendRequestsToBrokers(controllerContext.epoch) - } catch { - case e: IllegalStateException => - handleIllegalState(e) - } - stateChangeLog.info(s"Sent LeaderAndIsr request $updatedLeaderIsrAndControllerEpoch with " + - s"new replica assignment $assignment to leader ${updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader} " + - s"for partition being reassigned $topicPartition") - - case None => // fail the reassignment - stateChangeLog.error(s"Failed to send LeaderAndIsr request with new replica assignment " + - s"$assignment to leader for partition being reassigned $topicPartition") - } - } - - private def registerPartitionModificationsHandlers(topics: Seq[String]): Unit = { - topics.foreach { topic => - val partitionModificationsHandler = new PartitionModificationsHandler(eventManager, topic) - partitionModificationsHandlers.put(topic, partitionModificationsHandler) - } - partitionModificationsHandlers.values.foreach(zkClient.registerZNodeChangeHandler) - } - - private[controller] def unregisterPartitionModificationsHandlers(topics: Seq[String]): Unit = { - topics.foreach { topic => - partitionModificationsHandlers.remove(topic).foreach(handler => zkClient.unregisterZNodeChangeHandler(handler.path)) - } - } - - private def removePartitionFromReassigningPartitions(topicPartition: TopicPartition, - assignment: ReplicaAssignment): Unit = { - if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) { - maybeRemoveFromZkReassignment((tp, replicas) => tp == topicPartition && replicas == assignment.replicas) - controllerContext.partitionsBeingReassigned.remove(topicPartition) - } else { - throw new IllegalStateException("Cannot remove a reassigning partition because it is not present in memory") - } - } - - /** - * Remove partitions from an active zk-based reassignment (if one exists). - * - * @param shouldRemoveReassignment Predicate indicating which partition reassignments should be removed - */ - private def maybeRemoveFromZkReassignment(shouldRemoveReassignment: (TopicPartition, Seq[Int]) => Boolean): Unit = { - if (!zkClient.reassignPartitionsInProgress) - return - - val reassigningPartitions = zkClient.getPartitionReassignment - val (removingPartitions, updatedPartitionsBeingReassigned) = reassigningPartitions.partition { case (tp, replicas) => - shouldRemoveReassignment(tp, replicas) - } - info(s"Removing partitions $removingPartitions from the list of reassigned partitions in zookeeper") - - // write the new list to zookeeper - if (updatedPartitionsBeingReassigned.isEmpty) { - info(s"No more partitions need to be reassigned. Deleting zk path ${ReassignPartitionsZNode.path}") - zkClient.deletePartitionReassignment(controllerContext.epochZkVersion) - // Ensure we detect future reassignments - eventManager.put(ZkPartitionReassignment) - } else { - try { - zkClient.setOrCreatePartitionReassignment(updatedPartitionsBeingReassigned, controllerContext.epochZkVersion) - } catch { - case e: ZooKeeperClientException => throw new AdminOperationException(e) - } - } - } - - private def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicPartition], - isTriggeredByAutoRebalance : Boolean): Unit = { - for (partition <- partitionsToBeRemoved) { - // check the status - val currentLeader = controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader - val preferredReplica = controllerContext.partitionReplicaAssignment(partition).head - if (currentLeader == preferredReplica) { - info(s"Partition $partition completed preferred replica leader election. New leader is $preferredReplica") - } else { - warn(s"Partition $partition failed to complete preferred replica leader election to $preferredReplica. " + - s"Leader is still $currentLeader") - } - } - if (!isTriggeredByAutoRebalance) { - zkClient.deletePreferredReplicaElection(controllerContext.epochZkVersion) - // Ensure we detect future preferred replica leader elections - eventManager.put(ReplicaLeaderElection(None, ElectionType.PREFERRED, ZkTriggered)) - } - } - - /** - * Send the leader information for selected partitions to selected brokers so that they can correctly respond to - * metadata requests - * - * @param brokers The brokers that the update metadata request should be sent to - */ - private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = { - try { - brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) - brokerRequestBatch.sendRequestsToBrokers(epoch) - } catch { - case e: IllegalStateException => - handleIllegalState(e) - } - } - - /** - * Does not change leader or isr, but just increments the leader epoch - * - * @param partition partition - * @return the new leaderAndIsr with an incremented leader epoch, or None if leaderAndIsr is empty. - */ - private def updateLeaderEpoch(partition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { - debug(s"Updating leader epoch for partition $partition") - var finalLeaderIsrAndControllerEpoch: Option[LeaderIsrAndControllerEpoch] = None - var zkWriteCompleteOrUnnecessary = false - while (!zkWriteCompleteOrUnnecessary) { - // refresh leader and isr from zookeeper again - zkWriteCompleteOrUnnecessary = zkClient.getTopicPartitionStates(Seq(partition)).get(partition) match { - case Some(leaderIsrAndControllerEpoch) => - val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - val controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - if (controllerEpoch > epoch) - throw new StateChangeFailedException("Leader and isr path written by another controller. This probably " + - s"means the current controller with epoch $epoch went through a soft failure and another " + - s"controller was elected with epoch $controllerEpoch. Aborting state change by this controller") - // increment the leader epoch even if there are no leader or isr changes to allow the leader to cache the expanded - // assigned replica list - val newLeaderAndIsr = leaderAndIsr.newEpoch - // update the new leadership decision in zookeeper or retry - val UpdateLeaderAndIsrResult(finishedUpdates, _) = - zkClient.updateLeaderAndIsr(immutable.Map(partition -> newLeaderAndIsr), epoch, controllerContext.epochZkVersion) - - finishedUpdates.get(partition) match { - case Some(Right(leaderAndIsr)) => - val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, epoch) - controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) - finalLeaderIsrAndControllerEpoch = Some(leaderIsrAndControllerEpoch) - info(s"Updated leader epoch for partition $partition to ${leaderAndIsr.leaderEpoch}, zkVersion=${leaderAndIsr.partitionEpoch}") - true - case Some(Left(e)) => throw e - case None => false - } - case None => - throw new IllegalStateException(s"Cannot update leader epoch for partition $partition as " + - "leaderAndIsr path is empty. This could mean we somehow tried to reassign a partition that doesn't exist") - } - } - finalLeaderIsrAndControllerEpoch - } - - private def checkAndTriggerAutoLeaderRebalance(): Unit = { - trace("Checking need to trigger auto leader balancing") - val preferredReplicasForTopicsByBrokers: Map[Int, Map[TopicPartition, Seq[Int]]] = - controllerContext.allPartitions.filterNot { - tp => topicDeletionManager.isTopicQueuedUpForDeletion(tp.topic) - }.map { tp => - (tp, controllerContext.partitionReplicaAssignment(tp) ) - }.toMap.groupBy { case (_, assignedReplicas) => assignedReplicas.head } - - // for each broker, check if a preferred replica election needs to be triggered - preferredReplicasForTopicsByBrokers.foreachEntry { (leaderBroker, topicPartitionsForBroker) => - val topicsNotInPreferredReplica = topicPartitionsForBroker.filter { case (topicPartition, _) => - val leadershipInfo = controllerContext.partitionLeadershipInfo(topicPartition) - leadershipInfo.exists(_.leaderAndIsr.leader != leaderBroker) - } - debug(s"Topics not in preferred replica for broker $leaderBroker $topicsNotInPreferredReplica") - - val imbalanceRatio = topicsNotInPreferredReplica.size.toDouble / topicPartitionsForBroker.size - trace(s"Leader imbalance ratio for broker $leaderBroker is $imbalanceRatio") - - // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions - // that need to be on this broker - if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { - val candidatePartitions = topicsNotInPreferredReplica.keys.filter(tp => - !topicDeletionManager.isTopicQueuedUpForDeletion(tp.topic) && - controllerContext.allTopics.contains(tp.topic) && - canPreferredReplicaBeLeader(tp) - ) - onReplicaElection(candidatePartitions.toSet, ElectionType.PREFERRED, AutoTriggered) - } - } - } - - private def canPreferredReplicaBeLeader(tp: TopicPartition): Boolean = { - val assignment = controllerContext.partitionReplicaAssignment(tp) - val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, tp)) - val isr = controllerContext.partitionLeadershipInfo(tp).get.leaderAndIsr.isr.asScala.toSeq.map(_.toInt) - PartitionLeaderElectionAlgorithms - .preferredReplicaPartitionLeaderElection(assignment, isr, liveReplicas.toSet) - .nonEmpty - } - - private def processAutoPreferredReplicaLeaderElection(): Unit = { - if (!isActive) return - try { - info("Processing automatic preferred replica leader election") - checkAndTriggerAutoLeaderRebalance() - } finally { - scheduleAutoLeaderRebalanceTask(delay = config.leaderImbalanceCheckIntervalSeconds, unit = TimeUnit.SECONDS) - } - } - - private def processUncleanLeaderElectionEnable(): Unit = { - if (!isActive) return - info("Unclean leader election has been enabled by default") - partitionStateMachine.triggerOnlinePartitionStateChange() - } - - private def processTopicUncleanLeaderElectionEnable(topic: String): Unit = { - if (!isActive) return - info(s"Unclean leader election has been enabled for topic $topic") - partitionStateMachine.triggerOnlinePartitionStateChange(topic) - } - - private def processControlledShutdown(id: Int, brokerEpoch: Long, controlledShutdownCallback: Try[Set[TopicPartition]] => Unit): Unit = { - val controlledShutdownResult = Try { doControlledShutdown(id, brokerEpoch) } - controlledShutdownCallback(controlledShutdownResult) - } - - private def doControlledShutdown(id: Int, brokerEpoch: Long): Set[TopicPartition] = { - if (!isActive) { - throw new ControllerMovedException("Controller moved to another broker. Aborting controlled shutdown") - } - - // broker epoch in the request is unknown if the controller hasn't been upgraded to use KIP-380 - // so we will keep the previous behavior and don't reject the request - if (brokerEpoch != AbstractControlRequest.UNKNOWN_BROKER_EPOCH) { - val cachedBrokerEpoch = controllerContext.liveBrokerIdAndEpochs(id) - if (brokerEpoch < cachedBrokerEpoch) { - val stateBrokerEpochErrorMessage = "Received controlled shutdown request from an old broker epoch " + - s"$brokerEpoch for broker $id. Current broker epoch is $cachedBrokerEpoch." - info(stateBrokerEpochErrorMessage) - throw new StaleBrokerEpochException(stateBrokerEpochErrorMessage) - } - } - - info(s"Shutting down broker $id") - - if (!controllerContext.liveOrShuttingDownBrokerIds.contains(id)) - throw new BrokerNotAvailableException(s"Broker id $id does not exist.") - - controllerContext.shuttingDownBrokerIds.add(id) - debug(s"All shutting down brokers: ${controllerContext.shuttingDownBrokerIds.mkString(",")}") - debug(s"Live brokers: ${controllerContext.liveBrokerIds.mkString(",")}") - - val partitionsToActOn = controllerContext.partitionsOnBroker(id).filter { partition => - controllerContext.partitionReplicaAssignment(partition).size > 1 && - controllerContext.partitionLeadershipInfo(partition).isDefined && - !topicDeletionManager.isTopicQueuedUpForDeletion(partition.topic) - } - val (partitionsLedByBroker, partitionsFollowedByBroker) = partitionsToActOn.partition { partition => - controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader == id - } - partitionStateMachine.handleStateChanges(partitionsLedByBroker.toSeq, OnlinePartition, Some(ControlledShutdownPartitionLeaderElectionStrategy)) - try { - brokerRequestBatch.newBatch() - partitionsFollowedByBroker.foreach { partition => - brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), partition, deletePartition = false) - } - brokerRequestBatch.sendRequestsToBrokers(epoch) - } catch { - case e: IllegalStateException => - handleIllegalState(e) - } - // If the broker is a follower, updates the isr in ZK and notifies the current leader - replicaStateMachine.handleStateChanges(partitionsFollowedByBroker.map(partition => - PartitionAndReplica(partition, id)).toSeq, OfflineReplica) - trace(s"All leaders = ${controllerContext.partitionsLeadershipInfo.mkString(",")}") - controllerContext.partitionLeadersOnBroker(id) - } - - private def processUpdateMetadataResponseReceived(updateMetadataResponse: UpdateMetadataResponse, brokerId: Int): Unit = { - if (!isActive) return - - if (updateMetadataResponse.error != Errors.NONE) { - stateChangeLogger.error(s"Received error ${updateMetadataResponse.error} in UpdateMetadata " + - s"response $updateMetadataResponse from broker $brokerId") - } - } - - private def processLeaderAndIsrResponseReceived(leaderAndIsrResponse: LeaderAndIsrResponse, brokerId: Int): Unit = { - if (!isActive) return - - if (leaderAndIsrResponse.error != Errors.NONE) { - stateChangeLogger.error(s"Received error ${leaderAndIsrResponse.error} in LeaderAndIsr " + - s"response $leaderAndIsrResponse from broker $brokerId") - return - } - - val offlineReplicas = new ArrayBuffer[TopicPartition]() - val onlineReplicas = new ArrayBuffer[TopicPartition]() - - leaderAndIsrResponse.partitionErrors(controllerContext.topicNames.asJava).forEach{ case (tp, error) => - if (error.code() == Errors.KAFKA_STORAGE_ERROR.code) - offlineReplicas += tp - else if (error.code() == Errors.NONE.code) - onlineReplicas += tp - } - - val previousOfflineReplicas = controllerContext.replicasOnOfflineDirs.getOrElse(brokerId, Set.empty[TopicPartition]) - val currentOfflineReplicas = mutable.Set() ++= previousOfflineReplicas --= onlineReplicas ++= offlineReplicas - controllerContext.replicasOnOfflineDirs.put(brokerId, currentOfflineReplicas) - val newOfflineReplicas = currentOfflineReplicas.diff(previousOfflineReplicas) - - if (newOfflineReplicas.nonEmpty) { - stateChangeLogger.info(s"Mark replicas ${newOfflineReplicas.mkString(",")} on broker $brokerId as offline") - onReplicasBecomeOffline(newOfflineReplicas.map(PartitionAndReplica(_, brokerId))) - } - } - - private def processTopicDeletionStopReplicaResponseReceived(replicaId: Int, - requestError: Errors, - partitionErrors: Map[TopicPartition, Errors]): Unit = { - if (!isActive) return - debug(s"Delete topic callback invoked on StopReplica response received from broker $replicaId: " + - s"request error = $requestError, partition errors = $partitionErrors") - - val partitionsInError = if (requestError != Errors.NONE) - partitionErrors.keySet - else - partitionErrors.filter { case (_, error) => error != Errors.NONE }.keySet - - val replicasInError = partitionsInError.map(PartitionAndReplica(_, replicaId)) - // move all the failed replicas to ReplicaDeletionIneligible - topicDeletionManager.failReplicaDeletion(replicasInError) - if (replicasInError.size != partitionErrors.size) { - // some replicas could have been successfully deleted - val deletedReplicas = partitionErrors.keySet.diff(partitionsInError) - topicDeletionManager.completeReplicaDeletion(deletedReplicas.map(PartitionAndReplica(_, replicaId))) - } - } - - private def processStartup(): Unit = { - zkClient.registerZNodeChangeHandlerAndCheckExistence(controllerChangeHandler) - elect() - } - - private def updateMetrics(): Unit = { - if (isActive) { - offlinePartitionCount = controllerContext.offlinePartitionCount - preferredReplicaImbalanceCount = controllerContext.preferredReplicaImbalanceCount - globalTopicCount = controllerContext.allTopics.size - globalPartitionCount = controllerContext.partitionWithLeadersCount - topicsToDeleteCount = controllerContext.topicsToBeDeleted.size - replicasToDeleteCount = controllerContext.topicsToBeDeleted.map { topic => - // For each enqueued topic, count the number of replicas that are not yet deleted - controllerContext.replicasForTopic(topic).count { replica => - controllerContext.replicaState(replica) != ReplicaDeletionSuccessful - } - }.sum - ineligibleTopicsToDeleteCount = controllerContext.topicsIneligibleForDeletion.size - ineligibleReplicasToDeleteCount = controllerContext.topicsToBeDeleted.map { topic => - // For each enqueued topic, count the number of replicas that are ineligible - controllerContext.replicasForTopic(topic).count { replica => - controllerContext.replicaState(replica) == ReplicaDeletionIneligible - } - }.sum - activeBrokerCount = controllerContext.liveOrShuttingDownBrokerIds.size - } else { - offlinePartitionCount = 0 - preferredReplicaImbalanceCount = 0 - globalTopicCount = 0 - globalPartitionCount = 0 - topicsToDeleteCount = 0 - replicasToDeleteCount = 0 - ineligibleTopicsToDeleteCount = 0 - ineligibleReplicasToDeleteCount = 0 - activeBrokerCount = 0 - } - } - - // visible for testing - private[controller] def handleIllegalState(e: IllegalStateException): Nothing = { - // Resign if the controller is in an illegal state - error("Forcing the controller to resign") - brokerRequestBatch.clear() - triggerControllerMove() - throw e - } - - private def triggerControllerMove(): Unit = { - activeControllerId = zkClient.getControllerId.getOrElse(-1) - if (!isActive) { - warn("Controller has already moved when trying to trigger controller movement") - return - } - try { - val expectedControllerEpochZkVersion = controllerContext.epochZkVersion - activeControllerId = -1 - onControllerResignation() - zkClient.deleteController(expectedControllerEpochZkVersion) - } catch { - case _: ControllerMovedException => - warn("Controller has already moved when trying to trigger controller movement") - } - } - - private def maybeResign(): Unit = { - val wasActiveBeforeChange = isActive - zkClient.registerZNodeChangeHandlerAndCheckExistence(controllerChangeHandler) - activeControllerId = zkClient.getControllerId.getOrElse(-1) - if (wasActiveBeforeChange && !isActive) { - onControllerResignation() - } - } - - private def elect(): Unit = { - activeControllerId = zkClient.getControllerId.getOrElse(-1) - /* - * We can get here during the initial startup and the handleDeleted ZK callback. Because of the potential race condition, - * it's possible that the controller has already been elected when we get here. This check will prevent the following - * createEphemeralPath method from getting into an infinite loop if this broker is already the controller. - */ - if (activeControllerId != -1) { - debug(s"Broker $activeControllerId has been elected as the controller, so stopping the election process.") - return - } - - try { - val (epoch, epochZkVersion) = zkClient.registerControllerAndIncrementControllerEpoch(config.brokerId) - controllerContext.epoch = epoch - controllerContext.epochZkVersion = epochZkVersion - activeControllerId = config.brokerId - - info(s"${config.brokerId} successfully elected as the controller. Epoch incremented to ${controllerContext.epoch} " + - s"and epoch zk version is now ${controllerContext.epochZkVersion}") - - onControllerFailover() - } catch { - case e: ControllerMovedException => - maybeResign() - - if (activeControllerId != -1) - debug(s"Broker $activeControllerId was elected as controller instead of broker ${config.brokerId}", e) - else - warn("A controller has been elected but just resigned, this will result in another round of election", e) - case t: Throwable => - error(s"Error while electing or becoming controller on broker ${config.brokerId}. " + - s"Trigger controller movement immediately", t) - triggerControllerMove() - } - } - - /** - * Partitions the provided map of brokers and epochs into 2 new maps: - * - The first map contains only those brokers whose features were found to be compatible with - * the existing finalized features. - * - The second map contains only those brokers whose features were found to be incompatible with - * the existing finalized features. - * - * @param brokersAndEpochs the map to be partitioned - * @return two maps: first contains compatible brokers and second contains - * incompatible brokers as explained above - */ - private def partitionOnFeatureCompatibility(brokersAndEpochs: Map[Broker, Long]): (Map[Broker, Long], Map[Broker, Long]) = { - // There can not be any feature incompatibilities when the feature versioning system is disabled - // or when the finalized feature cache is empty. Otherwise, we check if the non-empty contents - // of the cache are compatible with the supported features of each broker. - brokersAndEpochs.partition { - case (broker, _) => - !featureCache.getFeatureOption.exists( - latestFinalizedFeatures => - BrokerFeatures.hasIncompatibleFeatures(broker.features, - latestFinalizedFeatures.finalizedFeatures().asScala. - map(kv => (kv._1, kv._2.toShort: java.lang.Short)).toMap.asJava)) - } - } - - private def processBrokerChange(): Unit = { - if (!isActive) return - val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster - val curBrokerIdAndEpochs = curBrokerAndEpochs map { case (broker, epoch) => (broker.id, epoch) } - val curBrokerIds = curBrokerIdAndEpochs.keySet - val liveOrShuttingDownBrokerIds = controllerContext.liveOrShuttingDownBrokerIds - val newBrokerIds = curBrokerIds.diff(liveOrShuttingDownBrokerIds) - val deadBrokerIds = liveOrShuttingDownBrokerIds.diff(curBrokerIds) - val bouncedBrokerIds = (curBrokerIds & liveOrShuttingDownBrokerIds) - .filter(brokerId => curBrokerIdAndEpochs(brokerId) > controllerContext.liveBrokerIdAndEpochs(brokerId)) - val newBrokerAndEpochs = curBrokerAndEpochs.filter { case (broker, _) => newBrokerIds.contains(broker.id) } - val bouncedBrokerAndEpochs = curBrokerAndEpochs.filter { case (broker, _) => bouncedBrokerIds.contains(broker.id) } - val newBrokerIdsSorted = newBrokerIds.toSeq.sorted - val deadBrokerIdsSorted = deadBrokerIds.toSeq.sorted - val liveBrokerIdsSorted = curBrokerIds.toSeq.sorted - val bouncedBrokerIdsSorted = bouncedBrokerIds.toSeq.sorted - info(s"Newly added brokers: ${newBrokerIdsSorted.mkString(",")}, " + - s"deleted brokers: ${deadBrokerIdsSorted.mkString(",")}, " + - s"bounced brokers: ${bouncedBrokerIdsSorted.mkString(",")}, " + - s"all live brokers: ${liveBrokerIdsSorted.mkString(",")}") - - newBrokerAndEpochs.keySet.foreach(controllerChannelManager.addBroker) - bouncedBrokerIds.foreach(controllerChannelManager.removeBroker) - bouncedBrokerAndEpochs.keySet.foreach(controllerChannelManager.addBroker) - deadBrokerIds.foreach(controllerChannelManager.removeBroker) - - if (newBrokerIds.nonEmpty) { - val (newCompatibleBrokerAndEpochs, newIncompatibleBrokerAndEpochs) = - partitionOnFeatureCompatibility(newBrokerAndEpochs) - if (newIncompatibleBrokerAndEpochs.nonEmpty) { - warn("Ignoring registration of new brokers due to incompatibilities with finalized features: " + - newIncompatibleBrokerAndEpochs.map { case (broker, _) => broker.id }.toSeq.sorted.mkString(",")) - } - controllerContext.addLiveBrokers(newCompatibleBrokerAndEpochs) - onBrokerStartup(newBrokerIdsSorted) - } - if (bouncedBrokerIds.nonEmpty) { - controllerContext.removeLiveBrokers(bouncedBrokerIds) - onBrokerFailure(bouncedBrokerIdsSorted) - val (bouncedCompatibleBrokerAndEpochs, bouncedIncompatibleBrokerAndEpochs) = - partitionOnFeatureCompatibility(bouncedBrokerAndEpochs) - if (bouncedIncompatibleBrokerAndEpochs.nonEmpty) { - warn("Ignoring registration of bounced brokers due to incompatibilities with finalized features: " + - bouncedIncompatibleBrokerAndEpochs.map { case (broker, _) => broker.id }.toSeq.sorted.mkString(",")) - } - controllerContext.addLiveBrokers(bouncedCompatibleBrokerAndEpochs) - onBrokerStartup(bouncedBrokerIdsSorted) - } - if (deadBrokerIds.nonEmpty) { - controllerContext.removeLiveBrokers(deadBrokerIds) - onBrokerFailure(deadBrokerIdsSorted) - } - - if (newBrokerIds.nonEmpty || deadBrokerIds.nonEmpty || bouncedBrokerIds.nonEmpty) { - info(s"Updated broker epochs cache: ${controllerContext.liveBrokerIdAndEpochs}") - } - } - - private def processBrokerModification(brokerId: Int): Unit = { - if (!isActive) return - val newMetadataOpt = zkClient.getBroker(brokerId) - val oldMetadataOpt = controllerContext.liveOrShuttingDownBroker(brokerId) - if (newMetadataOpt.nonEmpty && oldMetadataOpt.nonEmpty) { - val oldMetadata = oldMetadataOpt.get - val newMetadata = newMetadataOpt.get - if (newMetadata.endPoints != oldMetadata.endPoints || !oldMetadata.features.equals(newMetadata.features)) { - info(s"Updated broker metadata: $oldMetadata -> $newMetadata") - controllerContext.updateBrokerMetadata(oldMetadata, newMetadata) - onBrokerUpdate(brokerId) - } - } - } - - private def processTopicChange(): Unit = { - if (!isActive) return - val topics = zkClient.getAllTopicsInCluster(true) - val newTopics = topics -- controllerContext.allTopics - val deletedTopics = controllerContext.allTopics.diff(topics) - controllerContext.setAllTopics(topics) - - registerPartitionModificationsHandlers(newTopics.toSeq) - val addedPartitionReplicaAssignment = zkClient.getReplicaAssignmentAndTopicIdForTopics(newTopics) - deletedTopics.foreach(controllerContext.removeTopic) - processTopicIds(addedPartitionReplicaAssignment) - - addedPartitionReplicaAssignment.foreach { case TopicIdReplicaAssignment(_, _, newAssignments) => - newAssignments.foreach { case (topicAndPartition, newReplicaAssignment) => - controllerContext.updatePartitionFullReplicaAssignment(topicAndPartition, newReplicaAssignment) - } - } - info(s"New topics: [$newTopics], deleted topics: [$deletedTopics], new partition replica assignment " + - s"[$addedPartitionReplicaAssignment]") - if (addedPartitionReplicaAssignment.nonEmpty) { - val partitionAssignments = addedPartitionReplicaAssignment - .map { case TopicIdReplicaAssignment(_, _, partitionsReplicas) => partitionsReplicas.keySet } - .reduce((s1, s2) => s1.union(s2)) - onNewPartitionCreation(partitionAssignments) - } - } - - private def processTopicIds(topicIdAssignments: Set[TopicIdReplicaAssignment]): Unit = { - // Create topic IDs for topics missing them if we are using topic IDs - val updatedTopicIdAssignments = { - val (withTopicIds, withoutTopicIds) = topicIdAssignments.partition(_.topicId.isDefined) - withTopicIds ++ zkClient.setTopicIds(withoutTopicIds, controllerContext.epochZkVersion) - } - - // Add topic IDs to controller context - // If we don't have IBP 2.8, but are running 2.8 code, put any topic IDs from the ZNode in controller context - // This is to avoid losing topic IDs during operations like partition reassignments while the cluster is in a mixed state - updatedTopicIdAssignments.foreach { topicIdAssignment => - topicIdAssignment.topicId.foreach { topicId => - controllerContext.addTopicId(topicIdAssignment.topic, topicId) - } - } - } - - private def processLogDirEventNotification(): Unit = { - if (!isActive) return - val sequenceNumbers = zkClient.getAllLogDirEventNotifications - try { - val brokerIds = zkClient.getBrokerIdsFromLogDirEvents(sequenceNumbers) - onBrokerLogDirFailure(brokerIds) - } finally { - // delete processed children - zkClient.deleteLogDirEventNotifications(sequenceNumbers, controllerContext.epochZkVersion) - } - } - - private def processPartitionModifications(topic: String): Unit = { - def restorePartitionReplicaAssignment( - topic: String, - newPartitionReplicaAssignment: Map[TopicPartition, ReplicaAssignment] - ): Unit = { - info("Restoring the partition replica assignment for topic %s".format(topic)) - - val existingPartitions = zkClient.getChildren(TopicPartitionsZNode.path(topic)) - val existingPartitionReplicaAssignment = newPartitionReplicaAssignment - .filter(p => existingPartitions.contains(p._1.partition.toString)) - .map { case (tp, _) => - tp -> controllerContext.partitionFullReplicaAssignment(tp) - }.toMap - - zkClient.setTopicAssignment(topic, - controllerContext.topicIds.get(topic), - existingPartitionReplicaAssignment, - controllerContext.epochZkVersion) - } - - if (!isActive) return - val partitionReplicaAssignment = zkClient.getFullReplicaAssignmentForTopics(immutable.Set(topic)) - val partitionsToBeAdded = partitionReplicaAssignment.filter { case (topicPartition, _) => - controllerContext.partitionReplicaAssignment(topicPartition).isEmpty - } - - if (topicDeletionManager.isTopicQueuedUpForDeletion(topic)) { - if (partitionsToBeAdded.nonEmpty) { - warn("Skipping adding partitions %s for topic %s since it is currently being deleted" - .format(partitionsToBeAdded.map(_._1.partition).mkString(","), topic)) - - restorePartitionReplicaAssignment(topic, partitionReplicaAssignment) - } else { - // This can happen if existing partition replica assignment are restored to prevent increasing partition count during topic deletion - info("Ignoring partition change during topic deletion as no new partitions are added") - } - } else if (partitionsToBeAdded.nonEmpty) { - info(s"New partitions to be added $partitionsToBeAdded") - partitionsToBeAdded.foreachEntry { (topicPartition, assignedReplicas) => - controllerContext.updatePartitionFullReplicaAssignment(topicPartition, assignedReplicas) - } - onNewPartitionCreation(partitionsToBeAdded.keySet) - } - } - - private def processTopicDeletion(): Unit = { - if (!isActive) return - var topicsToBeDeleted = zkClient.getTopicDeletions.toSet - debug(s"Delete topics listener fired for topics ${topicsToBeDeleted.mkString(",")} to be deleted") - val nonExistentTopics = topicsToBeDeleted -- controllerContext.allTopics - if (nonExistentTopics.nonEmpty) { - warn(s"Ignoring request to delete non-existing topics ${nonExistentTopics.mkString(",")}") - zkClient.deleteTopicDeletions(nonExistentTopics.toSeq, controllerContext.epochZkVersion) - } - topicsToBeDeleted --= nonExistentTopics - if (config.deleteTopicEnable) { - if (topicsToBeDeleted.nonEmpty) { - info(s"Starting topic deletion for topics ${topicsToBeDeleted.mkString(",")}") - // mark topic ineligible for deletion if other state changes are in progress - topicsToBeDeleted.foreach { topic => - val partitionReassignmentInProgress = - controllerContext.partitionsBeingReassigned.map(_.topic).contains(topic) - if (partitionReassignmentInProgress) - topicDeletionManager.markTopicIneligibleForDeletion(Set(topic), - reason = "topic reassignment in progress") - } - // add topic to deletion list - topicDeletionManager.enqueueTopicsForDeletion(topicsToBeDeleted) - } - } else { - // If delete topic is disabled remove entries under zookeeper path : /admin/delete_topics - info(s"Removing $topicsToBeDeleted since delete topic is disabled") - zkClient.deleteTopicDeletions(topicsToBeDeleted.toSeq, controllerContext.epochZkVersion) - } - } - - private def processZkPartitionReassignment(): Set[TopicPartition] = { - // We need to register the watcher if the path doesn't exist in order to detect future - // reassignments and we get the `path exists` check for free - if (isActive && zkClient.registerZNodeChangeHandlerAndCheckExistence(partitionReassignmentHandler)) { - val reassignmentResults = mutable.Map.empty[TopicPartition, ApiError] - val partitionsToReassign = mutable.Map.empty[TopicPartition, ReplicaAssignment] - - zkClient.getPartitionReassignment.foreachEntry { (tp, targetReplicas) => - maybeBuildReassignment(tp, Some(targetReplicas)) match { - case Some(context) => partitionsToReassign.put(tp, context) - case None => reassignmentResults.put(tp, new ApiError(Errors.NO_REASSIGNMENT_IN_PROGRESS)) - } - } - - reassignmentResults ++= maybeTriggerPartitionReassignment(partitionsToReassign) - val (partitionsReassigned, partitionsFailed) = reassignmentResults.partition(_._2.error == Errors.NONE) - if (partitionsFailed.nonEmpty) { - warn(s"Failed reassignment through zk with the following errors: $partitionsFailed") - maybeRemoveFromZkReassignment((tp, _) => partitionsFailed.contains(tp)) - } - partitionsReassigned.keySet - } else { - Set.empty - } - } - - /** - * Process a partition reassignment from the AlterPartitionReassignment API. If there is an - * existing reassignment through zookeeper for any of the requested partitions, they will be - * cancelled prior to beginning the new reassignment. Any zk-based reassignment for partitions - * which are NOT included in this call will not be affected. - * - * @param reassignments Map of reassignments passed through the AlterReassignments API. A null value - * means that we should cancel an in-progress reassignment. - * @param callback Callback to send AlterReassignments response - */ - private def processApiPartitionReassignment(reassignments: Map[TopicPartition, Option[Seq[Int]]], - callback: AlterReassignmentsCallback): Unit = { - if (!isActive) { - callback(Right(new ApiError(Errors.NOT_CONTROLLER))) - } else { - val reassignmentResults = mutable.Map.empty[TopicPartition, ApiError] - val partitionsToReassign = mutable.Map.empty[TopicPartition, ReplicaAssignment] - - reassignments.foreachEntry { (tp, targetReplicas) => - val maybeApiError = targetReplicas.flatMap(validateReplicas(tp, _)) - maybeApiError match { - case None => - maybeBuildReassignment(tp, targetReplicas) match { - case Some(context) => partitionsToReassign.put(tp, context) - case None => reassignmentResults.put(tp, new ApiError(Errors.NO_REASSIGNMENT_IN_PROGRESS)) - } - case Some(err) => - reassignmentResults.put(tp, err) - } - } - - // The latest reassignment (whether by API or through zk) always takes precedence, - // so remove from active zk reassignment (if one exists) - maybeRemoveFromZkReassignment((tp, _) => partitionsToReassign.contains(tp)) - - reassignmentResults ++= maybeTriggerPartitionReassignment(partitionsToReassign) - callback(Left(reassignmentResults)) - } - } - - private def validateReplicas(topicPartition: TopicPartition, replicas: Seq[Int]): Option[ApiError] = { - val replicaSet = replicas.toSet - if (replicas.isEmpty) - Some(new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, - s"Empty replica list specified in partition reassignment.")) - else if (replicas.size != replicaSet.size) { - Some(new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, - s"Duplicate replica ids in partition reassignment replica list: $replicas")) - } else if (replicas.exists(_ < 0)) - Some(new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, - s"Invalid broker id in replica list: $replicas")) - else { - // Ensure that any new replicas are among the live brokers - val currentAssignment = controllerContext.partitionFullReplicaAssignment(topicPartition) - val newAssignment = currentAssignment.reassignTo(replicas) - val areNewReplicasAlive = newAssignment.addingReplicas.toSet.subsetOf(controllerContext.liveBrokerIds) - if (!areNewReplicasAlive) - Some(new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, - s"Replica assignment has brokers that are not alive. Replica list: " + - s"${newAssignment.addingReplicas}, live broker list: ${controllerContext.liveBrokerIds}")) - else None - } - } - - private def maybeBuildReassignment(topicPartition: TopicPartition, - targetReplicasOpt: Option[Seq[Int]]): Option[ReplicaAssignment] = { - val replicaAssignment = controllerContext.partitionFullReplicaAssignment(topicPartition) - if (replicaAssignment.isBeingReassigned) { - val targetReplicas = targetReplicasOpt.getOrElse(replicaAssignment.originReplicas) - Some(replicaAssignment.reassignTo(targetReplicas)) - } else { - targetReplicasOpt.map { targetReplicas => - replicaAssignment.reassignTo(targetReplicas) - } - } - } - - private def processPartitionReassignmentIsrChange(topicPartition: TopicPartition): Unit = { - if (!isActive) return - - if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) { - maybeCompleteReassignment(topicPartition) - } - } - - private def maybeCompleteReassignment(topicPartition: TopicPartition): Unit = { - val reassignment = controllerContext.partitionFullReplicaAssignment(topicPartition) - if (isReassignmentComplete(topicPartition, reassignment)) { - // resume the partition reassignment process - info(s"Target replicas ${reassignment.targetReplicas} have all caught up with the leader for " + - s"reassigning partition $topicPartition") - onPartitionReassignment(topicPartition, reassignment) - } - } - - private def processListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]], callback: ListReassignmentsCallback): Unit = { - if (!isActive) { - callback(Right(new ApiError(Errors.NOT_CONTROLLER))) - } else { - val results: mutable.Map[TopicPartition, ReplicaAssignment] = mutable.Map.empty - val partitionsToList = partitionsOpt match { - case Some(partitions) => partitions - case None => controllerContext.partitionsBeingReassigned - } - - partitionsToList.foreach { tp => - val assignment = controllerContext.partitionFullReplicaAssignment(tp) - if (assignment.isBeingReassigned) { - results += tp -> assignment - } - } - - callback(Left(results)) - } - } - - /** - * Returns the new finalized version for the feature, if there are no feature - * incompatibilities seen with all known brokers for the provided feature update. - * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST. - * - * @param update the feature update to be processed (this can not be meant to delete the feature) - * - * @return the new finalized version or error, as described above. - */ - private def newFinalizedVersionOrIncompatibilityError(update: UpdateFeaturesRequest.FeatureUpdateItem): - Either[Short, ApiError] = { - if (update.isDeleteRequest) { - throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update") - } - - val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature) - if (supportedVersionRange == null) { - Right(new ApiError(Errors.INVALID_REQUEST, - "Could not apply finalized feature update because the provided feature" + - " is not supported.")) - } else { - val newVersion = update.versionLevel() - if (supportedVersionRange.isIncompatibleWith(newVersion)) { - Right(new ApiError(Errors.INVALID_REQUEST, - "Could not apply finalized feature update because the provided" + - s" versionLevel:${update.versionLevel} is lower than the" + - s" supported minVersion:${supportedVersionRange.min}.")) - } else { - val newFinalizedFeature = Utils.mkMap(Utils.mkEntry(update.feature, newVersion: java.lang.Short)) - val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { - BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature) - }) - if (numIncompatibleBrokers == 0) { - Left(newVersion) - } else { - Right(new ApiError(Errors.INVALID_REQUEST, - "Could not apply finalized feature update because" + - " brokers were found to have incompatible versions for the feature.")) - } - } - } - } - - /** - * Validates a feature update on an existing finalized version. - * If the validation succeeds, then, the return value contains: - * 1. the new finalized version for the feature, if the feature update was not meant to delete the feature. - * 2. Option.empty, if the feature update was meant to delete the feature. - * - * If the validation fails, then returned value contains a suitable ApiError. - * - * @param update the feature update to be processed. - * @param existingVersion the existing finalized version which can be empty when no - * finalized version exists for the associated feature - * - * @return the new finalized version to be updated into ZK or error - * as described above. - */ - private def validateFeatureUpdate(update: UpdateFeaturesRequest.FeatureUpdateItem, - existingVersion: Option[Short]): Either[Option[Short], ApiError] = { - def newVersionRangeOrError(update: UpdateFeaturesRequest.FeatureUpdateItem): Either[Option[Short], ApiError] = { - newFinalizedVersionOrIncompatibilityError(update) - .fold(versionRange => Left(Some(versionRange)), error => Right(error)) - } - - if (update.feature.isEmpty) { - // Check that the feature name is not empty. - Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty.")) - } else if (update.upgradeType.equals(UpgradeType.UNKNOWN)) { - Right(new ApiError(Errors.INVALID_REQUEST, "Received unknown upgrade type.")) - } else { - - // We handle deletion requests separately from non-deletion requests. - if (update.isDeleteRequest) { - if (existingVersion.isEmpty) { - // Disallow deletion of a non-existing finalized feature. - Right(new ApiError(Errors.INVALID_REQUEST, - "Can not delete non-existing finalized feature.")) - } else { - Left(Option.empty) - } - } else if (update.versionLevel() < 1) { - // Disallow deletion of a finalized feature without SAFE downgrade type. - Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not provide versionLevel: ${update.versionLevel} less" + - s" than 1 without setting the SAFE downgradeType in the request.")) - } else { - existingVersion.map(existing => - if (update.versionLevel == existing) { - // Disallow a case where target versionLevel matches existing versionLevel. - Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not ${if (update.upgradeType.equals(UpgradeType.SAFE_DOWNGRADE)) "downgrade" else "upgrade"}" + - s" a finalized feature from existing versionLevel:$existing" + - " to the same value.")) - } else if (update.versionLevel < existing && !update.upgradeType.equals(UpgradeType.SAFE_DOWNGRADE)) { - // Disallow downgrade of a finalized feature without the downgradeType set. - Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature from existing" + - s" versionLevel:$existing to provided" + - s" versionLevel:${update.versionLevel} without setting the" + - " downgradeType to SAFE in the request.")) - } else if (!update.upgradeType.equals(UpgradeType.UPGRADE) && update.versionLevel > existing) { - // Disallow a request that sets downgradeType without specifying a - // versionLevel that's lower than the existing versionLevel. - Right(new ApiError(Errors.INVALID_REQUEST, - s"When the downgradeType is set to SAFE in the request, the provided" + - s" versionLevel:${update.versionLevel} can not be greater than" + - s" existing versionLevel:$existing.")) - } else { - newVersionRangeOrError(update) - } - ).getOrElse(newVersionRangeOrError(update)) - } - } - } - - private def processFeatureUpdates(request: UpdateFeaturesRequest, - callback: UpdateFeaturesCallback): Unit = { - if (isActive) { - processFeatureUpdatesWithActiveController(request, callback) - } else { - callback(Left(new ApiError(Errors.NOT_CONTROLLER))) - } - } - - private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest, - callback: UpdateFeaturesCallback): Unit = { - val updates = request.featureUpdates - val existingFeatures = featureCache.getFeatureOption - .map(featuresAndEpoch => featuresAndEpoch.finalizedFeatures().asScala.map(kv => (kv._1, kv._2.toShort)).toMap) - .getOrElse(Map[String, Short]()) - // A map with key being feature name and value being finalized version. - // This contains the target features to be eventually written to FeatureZNode. - val targetFeatures = scala.collection.mutable.Map[String, Short]() ++ existingFeatures - // A map with key being feature name and value being error encountered when the FeatureUpdate - // was applied. - val errors = scala.collection.mutable.Map[String, ApiError]() - - // Below we process each FeatureUpdate using the following logic: - // - If a FeatureUpdate is found to be valid, then: - // - The corresponding entry in errors map would be updated to contain Errors.NONE. - // - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated - // to contain the new finalized version for the feature. - // - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the - // targetFeatures map. - // - Otherwise if a FeatureUpdate is found to be invalid, then: - // - The corresponding entry in errors map would be updated with the appropriate ApiError. - // - The entry in targetFeatures map is left untouched. - updates.asScala.iterator.foreach { update => - validateFeatureUpdate(update, existingFeatures.get(update.feature())) match { - case Left(newVersionRangeOrNone) => - newVersionRangeOrNone match { - case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange) - case None => targetFeatures -= update.feature() - } - errors += (update.feature() -> new ApiError(Errors.NONE)) - case Right(featureUpdateFailureReason) => - errors += (update.feature() -> featureUpdateFailureReason) - } - } - - // If the existing and target features are the same, then, we skip the update to the - // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents - // of the FeatureZNode with the new features. This may result in partial or full modification - // of the existing finalized features in ZK. - try { - if (!existingFeatures.equals(targetFeatures)) { - val newNode = FeatureZNode(config.interBrokerProtocolVersion, FeatureZNodeStatus.Enabled, targetFeatures) - val newVersion = updateFeatureZNode(newNode) - featureCache.waitUntilFeatureEpochOrThrow(newVersion, request.data().timeoutMs()) - } - } catch { - // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE), - // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed - // for these. For the rest, the existing error is left untouched. - case e: Exception => - warn(s"Processing of feature updates: $request failed due to error: $e") - errors.foreach { case (feature, apiError) => - if (apiError.error() == Errors.NONE) { - errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED) - } - } - } finally { - callback(Right(errors)) - } - } - - private def processIsrChangeNotification(): Unit = { - def processUpdateNotifications(partitions: Seq[TopicPartition]): Unit = { - val liveBrokers: Seq[Int] = controllerContext.liveOrShuttingDownBrokerIds.toSeq - debug(s"Sending MetadataRequest to Brokers: $liveBrokers for TopicPartitions: $partitions") - sendUpdateMetadataRequest(liveBrokers, partitions.toSet) - } - - if (!isActive) return - val sequenceNumbers = zkClient.getAllIsrChangeNotifications - try { - val partitions = zkClient.getPartitionsFromIsrChangeNotifications(sequenceNumbers) - if (partitions.nonEmpty) { - updateLeaderAndIsrCache(partitions) - processUpdateNotifications(partitions) - - // During a partial upgrade, the controller may be on an IBP which assumes - // ISR changes through the `AlterPartition` API while some brokers are on an older - // IBP which assumes notification through Zookeeper. In this case, since the - // controller will not have registered watches for reassigning partitions, we - // can still rely on the batch ISR change notification path in order to - // complete the reassignment. - partitions.filter(controllerContext.partitionsBeingReassigned.contains).foreach { topicPartition => - maybeCompleteReassignment(topicPartition) - } - } - } finally { - // delete the notifications - zkClient.deleteIsrChangeNotifications(sequenceNumbers, controllerContext.epochZkVersion) - } - } - - def electLeaders( - partitions: Set[TopicPartition], - electionType: ElectionType, - callback: ElectLeadersCallback - ): Unit = { - eventManager.put(ReplicaLeaderElection(Some(partitions), electionType, AdminClientTriggered, callback)) - } - - def listPartitionReassignments(partitions: Option[Set[TopicPartition]], - callback: ListReassignmentsCallback): Unit = { - eventManager.put(ListPartitionReassignments(partitions, callback)) - } - - def updateFeatures(request: UpdateFeaturesRequest, - callback: UpdateFeaturesCallback): Unit = { - eventManager.put(UpdateFeatures(request, callback)) - } - - def alterPartitionReassignments(partitions: Map[TopicPartition, Option[Seq[Int]]], - callback: AlterReassignmentsCallback): Unit = { - eventManager.put(ApiPartitionReassignment(partitions, callback)) - } - - private def processReplicaLeaderElection( - partitionsFromAdminClientOpt: Option[Set[TopicPartition]], - electionType: ElectionType, - electionTrigger: ElectionTrigger, - callback: ElectLeadersCallback - ): Unit = { - if (!isActive) { - callback(partitionsFromAdminClientOpt.fold(Map.empty[TopicPartition, Either[ApiError, Int]]) { partitions => - partitions.iterator.map(partition => partition -> Left(new ApiError(Errors.NOT_CONTROLLER, null))).toMap - }) - } else { - // We need to register the watcher if the path doesn't exist in order to detect future preferred replica - // leader elections and we get the `path exists` check for free - if (electionTrigger == AdminClientTriggered || zkClient.registerZNodeChangeHandlerAndCheckExistence(preferredReplicaElectionHandler)) { - val partitions = partitionsFromAdminClientOpt match { - case Some(partitions) => partitions - case None => zkClient.getPreferredReplicaElection - } - - val allPartitions = controllerContext.allPartitions - val (knownPartitions, unknownPartitions) = partitions.partition(tp => allPartitions.contains(tp)) - unknownPartitions.foreach { p => - info(s"Skipping replica leader election ($electionType) for partition $p by $electionTrigger since it doesn't exist.") - } - - val (partitionsBeingDeleted, livePartitions) = knownPartitions.partition(partition => - topicDeletionManager.isTopicQueuedUpForDeletion(partition.topic)) - if (partitionsBeingDeleted.nonEmpty) { - warn(s"Skipping replica leader election ($electionType) for partitions $partitionsBeingDeleted " + - s"by $electionTrigger since the respective topics are being deleted") - } - - // partition those that have a valid leader - val (electablePartitions, alreadyValidLeader) = livePartitions.partition { partition => - electionType match { - case ElectionType.PREFERRED => - val assignedReplicas = controllerContext.partitionReplicaAssignment(partition) - val preferredReplica = assignedReplicas.head - val currentLeader = controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader - currentLeader != preferredReplica - - case ElectionType.UNCLEAN => - val currentLeader = controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader - currentLeader == LeaderAndIsr.NO_LEADER || !controllerContext.isLiveBroker(currentLeader) - } - } - - val results = onReplicaElection(electablePartitions, electionType, electionTrigger).map { - case (k, Left(ex)) => - if (ex.isInstanceOf[StateChangeFailedException]) { - val error = if (electionType == ElectionType.PREFERRED) { - Errors.PREFERRED_LEADER_NOT_AVAILABLE - } else { - Errors.ELIGIBLE_LEADERS_NOT_AVAILABLE - } - k -> Left(new ApiError(error, ex.getMessage)) - } else { - k -> Left(ApiError.fromThrowable(ex)) - } - case (k, Right(leaderAndIsr)) => k -> Right(leaderAndIsr.leader) - } ++ - alreadyValidLeader.map(_ -> Left(new ApiError(Errors.ELECTION_NOT_NEEDED))) ++ - partitionsBeingDeleted.map( - _ -> Left(new ApiError(Errors.INVALID_TOPIC_EXCEPTION, "The topic is being deleted")) - ) ++ - unknownPartitions.map( - _ -> Left(new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, "The partition does not exist.")) - ) - - debug(s"Waiting for any successful result for election type ($electionType) by $electionTrigger for partitions: $results") - callback(results) - } - } - } - - def alterPartitions( - alterPartitionRequest: AlterPartitionRequestData, - alterPartitionRequestVersion: Short, - callback: AlterPartitionResponseData => Unit - ): Unit = { - eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - alterPartitionRequestVersion, - callback - )) - } - - private def processAlterPartition( - alterPartitionRequest: AlterPartitionRequestData, - alterPartitionRequestVersion: Short, - callback: AlterPartitionResponseData => Unit - ): Unit = { - val partitionResponses = try { - tryProcessAlterPartition( - alterPartitionRequest, - alterPartitionRequestVersion, - callback - ) - } catch { - case e: Throwable => - error(s"Error when processing AlterPartition: $alterPartitionRequest", e) - callback(new AlterPartitionResponseData().setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code)) - mutable.Map.empty - } - - // After we have returned the result of the `AlterPartition` request, we should check whether - // there are any reassignments which can be completed by a successful ISR expansion. - partitionResponses.foreachEntry { (topicPartition, partitionResponse) => - if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) { - val isSuccessfulUpdate = partitionResponse.isRight - if (isSuccessfulUpdate) { - maybeCompleteReassignment(topicPartition) - } - } - } - } - - private def tryProcessAlterPartition( - alterPartitionRequest: AlterPartitionRequestData, - alterPartitionRequestVersion: Short, - callback: AlterPartitionResponseData => Unit - ): mutable.Map[TopicPartition, Either[Errors, LeaderAndIsr]] = { - val useTopicsIds = alterPartitionRequestVersion > 1 - - // Handle a few short-circuits - if (!isActive) { - callback(new AlterPartitionResponseData().setErrorCode(Errors.NOT_CONTROLLER.code)) - return mutable.Map.empty - } - - val brokerId = alterPartitionRequest.brokerId - val brokerEpoch = alterPartitionRequest.brokerEpoch - val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) - if (brokerEpochOpt.isEmpty) { - info(s"Ignoring AlterPartition due to unknown broker $brokerId") - callback(new AlterPartitionResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)) - return mutable.Map.empty - } - - if (!brokerEpochOpt.contains(brokerEpoch)) { - info(s"Ignoring AlterPartition due to stale broker epoch $brokerEpoch and local broker epoch $brokerEpochOpt for broker $brokerId") - callback(new AlterPartitionResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)) - return mutable.Map.empty - } - - val partitionsToAlter = new mutable.HashMap[TopicPartition, LeaderAndIsr]() - val alterPartitionResponse = new AlterPartitionResponseData() - - alterPartitionRequest.topics.forEach { topicReq => - val topicNameOpt = if (useTopicsIds) { - controllerContext.topicName(topicReq.topicId) - } else { - Some(topicReq.topicName) - } - - topicNameOpt match { - case None => - val topicResponse = new AlterPartitionResponseData.TopicData() - .setTopicId(topicReq.topicId) - alterPartitionResponse.topics.add(topicResponse) - topicReq.partitions.forEach { partitionReq => - topicResponse.partitions.add(new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(partitionReq.partitionIndex) - .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code)) - } - - case Some(topicName) => - topicReq.partitions.forEach { partitionReq => - val isr = if (alterPartitionRequestVersion >= 3) { - partitionReq.newIsrWithEpochs.asScala.toList.map(brokerState => Integer.valueOf(brokerState.brokerId())).asJava - } else { - partitionReq.newIsr - } - partitionsToAlter.put( - new TopicPartition(topicName, partitionReq.partitionIndex), - new LeaderAndIsr( - alterPartitionRequest.brokerId, - partitionReq.leaderEpoch, - isr, - LeaderRecoveryState.of(partitionReq.leaderRecoveryState), - partitionReq.partitionEpoch - ) - ) - } - } - } - - val partitionResponses = mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]() - // Determine which partitions we will accept the new ISR for - val adjustedIsrs = partitionsToAlter.flatMap { case (tp, newLeaderAndIsr) => - controllerContext.partitionLeadershipInfo(tp) match { - case Some(leaderIsrAndControllerEpoch) => - val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (newLeaderAndIsr.partitionEpoch > currentLeaderAndIsr.partitionEpoch - || newLeaderAndIsr.leaderEpoch > currentLeaderAndIsr.leaderEpoch) { - // If the partition leader has a higher partition/leader epoch, then it is likely - // that this node is no longer the active controller. We return NOT_CONTROLLER in - // this case to give the leader an opportunity to find the new controller. - partitionResponses(tp) = Left(Errors.NOT_CONTROLLER) - None - } else if (newLeaderAndIsr.leaderEpoch < currentLeaderAndIsr.leaderEpoch) { - partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) - None - } else if (newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) { - // If a partition is already in the desired state, just return it - // this check must be done before fencing based on partition epoch to maintain idempotency - partitionResponses(tp) = Right(currentLeaderAndIsr) - None - } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) { - partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) - None - } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.size() > 1) { - partitionResponses(tp) = Left(Errors.INVALID_REQUEST) - info( - s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " + - s"$newLeaderAndIsr" - ) - None - } else if (currentLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERED && - newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING) { - - partitionResponses(tp) = Left(Errors.INVALID_REQUEST) - info( - s"Rejecting AlterPartition from node $brokerId for $tp because the leader recovery state cannot change from " + - s"RECOVERED to RECOVERING: $newLeaderAndIsr" - ) - None - } else { - // Pull out replicas being added to ISR and verify they are all online. - // If a replica is not online, reject the update as specified in KIP-841. - val ineligibleReplicas = newLeaderAndIsr.isr.asScala.toSet.map(Int.unbox) -- controllerContext.liveBrokerIds - if (ineligibleReplicas.nonEmpty) { - info(s"Rejecting AlterPartition request from node $brokerId for $tp because " + - s"it specified ineligible replicas $ineligibleReplicas in the new ISR ${newLeaderAndIsr.isr}." - ) - - if (alterPartitionRequestVersion > 1) { - partitionResponses(tp) = Left(Errors.INELIGIBLE_REPLICA) - } else { - partitionResponses(tp) = Left(Errors.OPERATION_NOT_ATTEMPTED) - } - None - } else { - Some(tp -> newLeaderAndIsr) - } - } - - case None => - partitionResponses(tp) = Left(Errors.UNKNOWN_TOPIC_OR_PARTITION) - None - } - } - - // Do the updates in ZK - debug(s"Updating ISRs for partitions: ${adjustedIsrs.keySet}.") - val UpdateLeaderAndIsrResult(finishedUpdates, badVersionUpdates) = zkClient.updateLeaderAndIsr( - adjustedIsrs, controllerContext.epoch, controllerContext.epochZkVersion) - - val successfulUpdates = finishedUpdates.flatMap { case (partition, isrOrError) => - isrOrError match { - case Right(updatedIsr) => - debug(s"ISR for partition $partition updated to $updatedIsr.") - partitionResponses(partition) = Right(updatedIsr) - Some(partition -> updatedIsr) - case Left(e) => - error(s"Failed to update ISR for partition $partition", e) - partitionResponses(partition) = Left(Errors.forException(e)) - None - } - } - - badVersionUpdates.foreach { partition => - info(s"Failed to update ISR to ${adjustedIsrs(partition)} for partition $partition, bad ZK version.") - partitionResponses(partition) = Left(Errors.INVALID_UPDATE_VERSION) - } - - // Update our cache and send out metadata updates - updateLeaderAndIsrCache(successfulUpdates.keys.toSeq) - sendUpdateMetadataRequest( - controllerContext.liveOrShuttingDownBrokerIds.toSeq, - partitionsToAlter.keySet - ) - - partitionResponses.groupBy(_._1.topic).foreachEntry { (topicName, partitionResponses) => - // Add each topic part to the response - val topicResponse = if (useTopicsIds) { - new AlterPartitionResponseData.TopicData() - .setTopicId(controllerContext.topicIds.getOrElse(topicName, Uuid.ZERO_UUID)) - } else { - new AlterPartitionResponseData.TopicData() - .setTopicName(topicName) - } - alterPartitionResponse.topics.add(topicResponse) - - partitionResponses.foreachEntry { (tp, errorOrIsr) => - // Add each partition part to the response (new ISR or error) - errorOrIsr match { - case Left(error) => - topicResponse.partitions.add( - new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setErrorCode(error.code)) - case Right(leaderAndIsr) => - /* Setting the LeaderRecoveryState field is always safe because it will always be the same - * as the value set in the request. For version 0, that is always the default RECOVERED - * which is ignored when serializing to version 0. For any other version, the - * LeaderRecoveryState field is supported. - */ - topicResponse.partitions.add( - new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderId(leaderAndIsr.leader) - .setLeaderEpoch(leaderAndIsr.leaderEpoch) - .setIsr(leaderAndIsr.isr) - .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) - .setPartitionEpoch(leaderAndIsr.partitionEpoch) - ) - } - } - } - - callback(alterPartitionResponse) - - partitionResponses - } - - def allocateProducerIds(allocateProducerIdsRequest: AllocateProducerIdsRequestData, - callback: AllocateProducerIdsResponseData => Unit): Unit = { - - def eventManagerCallback(results: Either[Errors, ProducerIdsBlock]): Unit = { - results match { - case Left(error) => callback.apply(new AllocateProducerIdsResponseData().setErrorCode(error.code)) - case Right(pidBlock) => callback.apply( - new AllocateProducerIdsResponseData() - .setProducerIdStart(pidBlock.firstProducerId()) - .setProducerIdLen(pidBlock.size())) - } - } - eventManager.put(AllocateProducerIds(allocateProducerIdsRequest.brokerId, - allocateProducerIdsRequest.brokerEpoch, eventManagerCallback)) - } - - private def processAllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, ProducerIdsBlock] => Unit): Unit = { - // Handle a few short-circuits - if (!isActive) { - callback.apply(Left(Errors.NOT_CONTROLLER)) - return - } - - val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) - if (brokerEpochOpt.isEmpty) { - warn(s"Ignoring AllocateProducerIds due to unknown broker $brokerId") - callback.apply(Left(Errors.BROKER_ID_NOT_REGISTERED)) - return - } - - if (!brokerEpochOpt.contains(brokerEpoch)) { - warn(s"Ignoring AllocateProducerIds due to stale broker epoch $brokerEpoch for broker $brokerId") - callback.apply(Left(Errors.STALE_BROKER_EPOCH)) - return - } - } - - private def processControllerChange(): Unit = { - maybeResign() - } - - private def processReelect(): Unit = { - maybeResign() - elect() - } - - private def processRegisterBrokerAndReelect(): Unit = { - _brokerEpoch = zkClient.registerBroker(brokerInfo) - processReelect() - } - - private def processExpire(): Unit = { - activeControllerId = -1 - onControllerResignation() - } - - - override def process(event: ControllerEvent): Unit = { - try { - event match { - case event: MockEvent => - // Used only in test cases - event.process() - case ShutdownEventThread => - error("Received a ShutdownEventThread event. This type of event is supposed to be handle by ControllerEventThread") - case AutoPreferredReplicaLeaderElection => - processAutoPreferredReplicaLeaderElection() - case ReplicaLeaderElection(partitions, electionType, electionTrigger, callback) => - processReplicaLeaderElection(partitions, electionType, electionTrigger, callback) - case UncleanLeaderElectionEnable => - processUncleanLeaderElectionEnable() - case TopicUncleanLeaderElectionEnable(topic) => - processTopicUncleanLeaderElectionEnable(topic) - case ControlledShutdown(id, brokerEpoch, callback) => - processControlledShutdown(id, brokerEpoch, callback) - case LeaderAndIsrResponseReceived(response, brokerId) => - processLeaderAndIsrResponseReceived(response, brokerId) - case UpdateMetadataResponseReceived(response, brokerId) => - processUpdateMetadataResponseReceived(response, brokerId) - case TopicDeletionStopReplicaResponseReceived(replicaId, requestError, partitionErrors) => - processTopicDeletionStopReplicaResponseReceived(replicaId, requestError, partitionErrors) - case BrokerChange => - processBrokerChange() - case BrokerModifications(brokerId) => - processBrokerModification(brokerId) - case ControllerChange => - processControllerChange() - case Reelect => - processReelect() - case RegisterBrokerAndReelect => - processRegisterBrokerAndReelect() - case Expire => - processExpire() - case TopicChange => - processTopicChange() - case LogDirEventNotification => - processLogDirEventNotification() - case PartitionModifications(topic) => - processPartitionModifications(topic) - case TopicDeletion => - processTopicDeletion() - case ApiPartitionReassignment(reassignments, callback) => - processApiPartitionReassignment(reassignments, callback) - case ZkPartitionReassignment => - processZkPartitionReassignment() - case ListPartitionReassignments(partitions, callback) => - processListPartitionReassignments(partitions, callback) - case UpdateFeatures(request, callback) => - processFeatureUpdates(request, callback) - case PartitionReassignmentIsrChange(partition) => - processPartitionReassignmentIsrChange(partition) - case IsrChangeNotification => - processIsrChangeNotification() - case AlterPartitionReceived(alterPartitionRequest, alterPartitionRequestVersion, callback) => - processAlterPartition(alterPartitionRequest, alterPartitionRequestVersion, callback) - case AllocateProducerIds(brokerId, brokerEpoch, callback) => - processAllocateProducerIds(brokerId, brokerEpoch, callback) - case Startup => - processStartup() - } - } catch { - case e: ControllerMovedException => - info(s"Controller moved to another broker when processing $event.", e) - maybeResign() - case e: Throwable => - error(s"Error processing event $event", e) - } finally { - updateMetrics() - } - } - - override def preempt(event: ControllerEvent): Unit = { - event.preempt() - } -} - -class BrokerChangeHandler(eventManager: ControllerEventManager) extends ZNodeChildChangeHandler { - override val path: String = BrokerIdsZNode.path - - override def handleChildChange(): Unit = { - eventManager.put(BrokerChange) - } -} - -class BrokerModificationsHandler(eventManager: ControllerEventManager, brokerId: Int) extends ZNodeChangeHandler { - override val path: String = BrokerIdZNode.path(brokerId) - - override def handleDataChange(): Unit = { - eventManager.put(BrokerModifications(brokerId)) - } -} - -class TopicChangeHandler(eventManager: ControllerEventManager) extends ZNodeChildChangeHandler { - override val path: String = TopicsZNode.path - - override def handleChildChange(): Unit = eventManager.put(TopicChange) -} - -class LogDirEventNotificationHandler(eventManager: ControllerEventManager) extends ZNodeChildChangeHandler { - override val path: String = LogDirEventNotificationZNode.path - - override def handleChildChange(): Unit = eventManager.put(LogDirEventNotification) -} - -class PartitionModificationsHandler(eventManager: ControllerEventManager, topic: String) extends ZNodeChangeHandler { - override val path: String = TopicZNode.path(topic) - - override def handleDataChange(): Unit = eventManager.put(PartitionModifications(topic)) -} - -class TopicDeletionHandler(eventManager: ControllerEventManager) extends ZNodeChildChangeHandler { - override val path: String = DeleteTopicsZNode.path - - override def handleChildChange(): Unit = eventManager.put(TopicDeletion) -} - -class PartitionReassignmentHandler(eventManager: ControllerEventManager) extends ZNodeChangeHandler { - override val path: String = ReassignPartitionsZNode.path - - // Note that the event is also enqueued when the znode is deleted, but we do it explicitly instead of relying on - // handleDeletion(). This approach is more robust as it doesn't depend on the watcher being re-registered after - // it's consumed during data changes (we ensure re-registration when the znode is deleted). - override def handleCreation(): Unit = eventManager.put(ZkPartitionReassignment) -} - -class PartitionReassignmentIsrChangeHandler(eventManager: ControllerEventManager, partition: TopicPartition) extends ZNodeChangeHandler { - override val path: String = TopicPartitionStateZNode.path(partition) - - override def handleDataChange(): Unit = eventManager.put(PartitionReassignmentIsrChange(partition)) -} - -class IsrChangeNotificationHandler(eventManager: ControllerEventManager) extends ZNodeChildChangeHandler { - override val path: String = IsrChangeNotificationZNode.path - - override def handleChildChange(): Unit = eventManager.put(IsrChangeNotification) -} - -object IsrChangeNotificationHandler { - val Version: Long = 1L -} - -class PreferredReplicaElectionHandler(eventManager: ControllerEventManager) extends ZNodeChangeHandler { - override val path: String = PreferredReplicaElectionZNode.path - - override def handleCreation(): Unit = eventManager.put(ReplicaLeaderElection(None, ElectionType.PREFERRED, ZkTriggered)) -} - -class ControllerChangeHandler(eventManager: ControllerEventManager) extends ZNodeChangeHandler { - override val path: String = ControllerZNode.path - - override def handleCreation(): Unit = eventManager.put(ControllerChange) - override def handleDeletion(): Unit = eventManager.put(Reelect) - override def handleDataChange(): Unit = eventManager.put(ControllerChange) -} - -case class PartitionAndReplica(topicPartition: TopicPartition, replica: Int) { - def topic: String = topicPartition.topic - def partition: Int = topicPartition.partition - - override def toString: String = { - s"[Topic=$topic,Partition=$partition,Replica=$replica]" - } -} - -case class LeaderIsrAndControllerEpoch(leaderAndIsr: LeaderAndIsr, controllerEpoch: Int) { - override def toString: String = { - val leaderAndIsrInfo = new StringBuilder - leaderAndIsrInfo.append("(Leader:" + leaderAndIsr.leader) - leaderAndIsrInfo.append(",ISR:" + leaderAndIsr.isr.asScala.mkString(",")) - leaderAndIsrInfo.append(",LeaderRecoveryState:" + leaderAndIsr.leaderRecoveryState) - leaderAndIsrInfo.append(",LeaderEpoch:" + leaderAndIsr.leaderEpoch) - leaderAndIsrInfo.append(",ZkVersion:" + leaderAndIsr.partitionEpoch) - leaderAndIsrInfo.append(",ControllerEpoch:" + controllerEpoch + ")") - leaderAndIsrInfo.toString() - } -} - -private[controller] class ControllerStats { - private val metricsGroup = new KafkaMetricsGroup(this.getClass) - - val uncleanLeaderElectionRate: Meter = metricsGroup.newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS) - - val rateAndTimeMetrics: Map[ControllerState, Timer] = ControllerState.values.flatMap { state => - state.rateAndTimeMetricName.map { metricName => - state -> metricsGroup.newTimer(metricName, TimeUnit.MILLISECONDS, TimeUnit.SECONDS) - } - }.toMap - - // For test. - def removeMetric(name: String): Unit = { - metricsGroup.removeMetric(name) - } -} - -sealed trait ControllerEvent { - def state: ControllerState - // preempt() is not executed by `ControllerEventThread` but by the main thread. - def preempt(): Unit -} - -case object ControllerChange extends ControllerEvent { - override def state: ControllerState = ControllerState.ControllerChange - override def preempt(): Unit = {} -} - -case object Reelect extends ControllerEvent { - override def state: ControllerState = ControllerState.ControllerChange - override def preempt(): Unit = {} -} - -case object RegisterBrokerAndReelect extends ControllerEvent { - override def state: ControllerState = ControllerState.ControllerChange - override def preempt(): Unit = {} -} - -case object Expire extends ControllerEvent { - override def state: ControllerState = ControllerState.ControllerChange - override def preempt(): Unit = {} -} - -case object ShutdownEventThread extends ControllerEvent { - override def state: ControllerState = ControllerState.ControllerShutdown - override def preempt(): Unit = {} -} - -case object AutoPreferredReplicaLeaderElection extends ControllerEvent { - override def state: ControllerState = ControllerState.AutoLeaderBalance - override def preempt(): Unit = {} -} - -case object UncleanLeaderElectionEnable extends ControllerEvent { - override def state: ControllerState = ControllerState.UncleanLeaderElectionEnable - override def preempt(): Unit = {} -} - -case class TopicUncleanLeaderElectionEnable(topic: String) extends ControllerEvent { - override def state: ControllerState = ControllerState.TopicUncleanLeaderElectionEnable - override def preempt(): Unit = {} -} - -case class ControlledShutdown(id: Int, brokerEpoch: Long, controlledShutdownCallback: Try[Set[TopicPartition]] => Unit) extends ControllerEvent { - override def state: ControllerState = ControllerState.ControlledShutdown - override def preempt(): Unit = controlledShutdownCallback(Failure(new ControllerMovedException("Controller moved to another broker"))) -} - -case class LeaderAndIsrResponseReceived(leaderAndIsrResponse: LeaderAndIsrResponse, brokerId: Int) extends ControllerEvent { - override def state: ControllerState = ControllerState.LeaderAndIsrResponseReceived - override def preempt(): Unit = {} -} - -case class UpdateMetadataResponseReceived(updateMetadataResponse: UpdateMetadataResponse, brokerId: Int) extends ControllerEvent { - override def state: ControllerState = ControllerState.UpdateMetadataResponseReceived - override def preempt(): Unit = {} -} - -case class TopicDeletionStopReplicaResponseReceived(replicaId: Int, - requestError: Errors, - partitionErrors: Map[TopicPartition, Errors]) extends ControllerEvent { - override def state: ControllerState = ControllerState.TopicDeletion - override def preempt(): Unit = {} -} - -case object Startup extends ControllerEvent { - override def state: ControllerState = ControllerState.ControllerChange - override def preempt(): Unit = {} -} - -case object BrokerChange extends ControllerEvent { - override def state: ControllerState = ControllerState.BrokerChange - override def preempt(): Unit = {} -} - -case class BrokerModifications(brokerId: Int) extends ControllerEvent { - override def state: ControllerState = ControllerState.BrokerChange - override def preempt(): Unit = {} -} - -case object TopicChange extends ControllerEvent { - override def state: ControllerState = ControllerState.TopicChange - override def preempt(): Unit = {} -} - -case object LogDirEventNotification extends ControllerEvent { - override def state: ControllerState = ControllerState.LogDirChange - override def preempt(): Unit = {} -} - -case class PartitionModifications(topic: String) extends ControllerEvent { - override def state: ControllerState = ControllerState.TopicChange - override def preempt(): Unit = {} -} - -case object TopicDeletion extends ControllerEvent { - override def state: ControllerState = ControllerState.TopicDeletion - override def preempt(): Unit = {} -} - -case object ZkPartitionReassignment extends ControllerEvent { - override def state: ControllerState = ControllerState.AlterPartitionReassignment - override def preempt(): Unit = {} -} - -case class ApiPartitionReassignment(reassignments: Map[TopicPartition, Option[Seq[Int]]], - callback: AlterReassignmentsCallback) extends ControllerEvent { - override def state: ControllerState = ControllerState.AlterPartitionReassignment - override def preempt(): Unit = callback(Right(new ApiError(Errors.NOT_CONTROLLER))) -} - -case class PartitionReassignmentIsrChange(partition: TopicPartition) extends ControllerEvent { - override def state: ControllerState = ControllerState.AlterPartitionReassignment - override def preempt(): Unit = {} -} - -case object IsrChangeNotification extends ControllerEvent { - override def state: ControllerState = ControllerState.IsrChange - override def preempt(): Unit = {} -} - -case class AlterPartitionReceived( - alterPartitionRequest: AlterPartitionRequestData, - alterPartitionRequestVersion: Short, - callback: AlterPartitionResponseData => Unit -) extends ControllerEvent { - override def state: ControllerState = ControllerState.IsrChange - override def preempt(): Unit = {} -} - -case class ReplicaLeaderElection( - partitionsFromAdminClientOpt: Option[Set[TopicPartition]], - electionType: ElectionType, - electionTrigger: ElectionTrigger, - callback: ElectLeadersCallback = _ => {} -) extends ControllerEvent { - override def state: ControllerState = ControllerState.ManualLeaderBalance - - override def preempt(): Unit = callback( - partitionsFromAdminClientOpt.fold(Map.empty[TopicPartition, Either[ApiError, Int]]) { partitions => - partitions.iterator.map(partition => partition -> Left(new ApiError(Errors.NOT_CONTROLLER, null))).toMap - } - ) -} - -/** - * @param partitionsOpt - an Optional set of partitions. If not present, all reassigning partitions are to be listed - */ -case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]], - callback: ListReassignmentsCallback) extends ControllerEvent { - override def state: ControllerState = ControllerState.ListPartitionReassignment - override def preempt(): Unit = callback(Right(new ApiError(Errors.NOT_CONTROLLER, null))) -} - -case class UpdateFeatures(request: UpdateFeaturesRequest, - callback: UpdateFeaturesCallback) extends ControllerEvent { - override def state: ControllerState = ControllerState.UpdateFeatures - override def preempt(): Unit = {} -} - -case class AllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, ProducerIdsBlock] => Unit) - extends ControllerEvent { - override def state: ControllerState = ControllerState.Idle - override def preempt(): Unit = {} -} - - -// Used only in test cases -abstract class MockEvent(val state: ControllerState) extends ControllerEvent { - def process(): Unit - def preempt(): Unit -} diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala deleted file mode 100755 index d03ac18a24022..0000000000000 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ /dev/null @@ -1,156 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.controller - -import kafka.server.KafkaConfig -import kafka.utils.Logging -import kafka.zk.KafkaZkClient -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.metadata.LeaderAndIsr - -import scala.collection.{Map, Seq} - -abstract class PartitionStateMachine(controllerContext: ControllerContext) extends Logging { - /** - * Invoked on successful controller election. - */ - def startup(): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Invoked on controller shutdown. - */ - def shutdown(): Unit = { - throw new UnsupportedOperationException() - } - - /** - * This API invokes the OnlinePartition state change on all partitions in either the NewPartition or OfflinePartition - * state. This is called on a successful controller election and on broker changes - */ - def triggerOnlinePartitionStateChange(): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - throw new UnsupportedOperationException() - } - - def triggerOnlinePartitionStateChange(topic: String): Unit = { - throw new UnsupportedOperationException() - } - - def handleStateChanges( - partitions: Seq[TopicPartition], - targetState: PartitionState - ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - throw new UnsupportedOperationException() - } - - def handleStateChanges( - partitions: Seq[TopicPartition], - targetState: PartitionState, - leaderElectionStrategy: Option[PartitionLeaderElectionStrategy] - ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] -} - -/** - * This class represents the state machine for partitions. It defines the states that a partition can be in, and - * transitions to move the partition to another legal state. The different states that a partition can be in are - - * 1. NonExistentPartition: This state indicates that the partition was either never created or was created and then - * deleted. Valid previous state, if one exists, is OfflinePartition - * 2. NewPartition : After creation, the partition is in the NewPartition state. In this state, the partition should have - * replicas assigned to it, but no leader/isr yet. Valid previous states are NonExistentPartition - * 3. OnlinePartition : Once a leader is elected for a partition, it is in the OnlinePartition state. - * Valid previous states are NewPartition/OfflinePartition - * 4. OfflinePartition : If, after successful leader election, the leader for partition dies, then the partition - * moves to the OfflinePartition state. Valid previous states are NewPartition/OnlinePartition - */ -class ZkPartitionStateMachine(config: KafkaConfig, - stateChangeLogger: StateChangeLogger, - controllerContext: ControllerContext, - zkClient: KafkaZkClient, - controllerBrokerRequestBatch: ControllerBrokerRequestBatch) - extends PartitionStateMachine(controllerContext) { - - private val controllerId = config.brokerId - this.logIdent = s"[PartitionStateMachine controllerId=$controllerId] " - - /** - * Try to change the state of the given partitions to the given targetState, using the given - * partitionLeaderElectionStrategyOpt if a leader election is required. - * @param partitions The partitions - * @param targetState The state - * @param partitionLeaderElectionStrategyOpt The leader election strategy if a leader election is required. - * @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. - */ - override def handleStateChanges( - partitions: Seq[TopicPartition], - targetState: PartitionState, - partitionLeaderElectionStrategyOpt: Option[PartitionLeaderElectionStrategy] - ): Map[TopicPartition, Either[Throwable, LeaderAndIsr]] = { - throw new UnsupportedOperationException() - } -} - -object PartitionLeaderElectionAlgorithms { - def offlinePartitionLeaderElection(assignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int], uncleanLeaderElectionEnabled: Boolean, controllerContext: ControllerContext): Option[Int] = { - throw new UnsupportedOperationException() - } - - def reassignPartitionLeaderElection(reassignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int]): Option[Int] = { - throw new UnsupportedOperationException() - } - - def preferredReplicaPartitionLeaderElection(assignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int]): Option[Int] = { - throw new UnsupportedOperationException() - } - - def controlledShutdownPartitionLeaderElection(assignment: Seq[Int], isr: Seq[Int], liveReplicas: Set[Int], shuttingDownBrokers: Set[Int]): Option[Int] = { - throw new UnsupportedOperationException() - } -} - -sealed trait PartitionLeaderElectionStrategy -final case class OfflinePartitionLeaderElectionStrategy(allowUnclean: Boolean) extends PartitionLeaderElectionStrategy -case object ReassignPartitionLeaderElectionStrategy extends PartitionLeaderElectionStrategy -case object PreferredReplicaPartitionLeaderElectionStrategy extends PartitionLeaderElectionStrategy -case object ControlledShutdownPartitionLeaderElectionStrategy extends PartitionLeaderElectionStrategy - -sealed trait PartitionState { - def state: Byte - def validPreviousStates: Set[PartitionState] -} - -case object NewPartition extends PartitionState { - val state: Byte = 0 - val validPreviousStates: Set[PartitionState] = Set(NonExistentPartition) -} - -case object OnlinePartition extends PartitionState { - val state: Byte = 1 - val validPreviousStates: Set[PartitionState] = Set(NewPartition, OnlinePartition, OfflinePartition) -} - -case object OfflinePartition extends PartitionState { - val state: Byte = 2 - val validPreviousStates: Set[PartitionState] = Set(NewPartition, OnlinePartition, OfflinePartition) -} - -case object NonExistentPartition extends PartitionState { - val state: Byte = 3 - val validPreviousStates: Set[PartitionState] = Set(OfflinePartition) -} diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala deleted file mode 100644 index 7a58f277d0f3a..0000000000000 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ /dev/null @@ -1,115 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. -*/ -package kafka.controller - -import kafka.server.KafkaConfig -import kafka.utils.Logging -import kafka.zk.KafkaZkClient - -import scala.collection.Seq - -abstract class ReplicaStateMachine(controllerContext: ControllerContext) extends Logging { - /** - * Invoked on successful controller election. - */ - def startup(): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Invoked on controller shutdown. - */ - def shutdown(): Unit = { - throw new UnsupportedOperationException() - } - - def handleStateChanges(replicas: Seq[PartitionAndReplica], targetState: ReplicaState): Unit -} - -/** - * This class represents the state machine for replicas. It defines the states that a replica can be in, and - * transitions to move the replica to another legal state. The different states that a replica can be in are - - * 1. NewReplica : The controller can create new replicas during partition reassignment. In this state, a - * replica can only get become follower state change request. Valid previous - * state is NonExistentReplica - * 2. OnlineReplica : Once a replica is started and part of the assigned replicas for its partition, it is in this - * state. In this state, it can get either become leader or become follower state change requests. - * Valid previous state are NewReplica, OnlineReplica, OfflineReplica and ReplicaDeletionIneligible - * 3. OfflineReplica : If a replica dies, it moves to this state. This happens when the broker hosting the replica - * is down. Valid previous state are NewReplica, OnlineReplica, OfflineReplica and ReplicaDeletionIneligible - * 4. ReplicaDeletionStarted: If replica deletion starts, it is moved to this state. Valid previous state is OfflineReplica - * 5. ReplicaDeletionSuccessful: If replica responds with no error code in response to a delete replica request, it is - * moved to this state. Valid previous state is ReplicaDeletionStarted - * 6. ReplicaDeletionIneligible: If replica deletion fails, it is moved to this state. Valid previous states are - * ReplicaDeletionStarted and OfflineReplica - * 7. NonExistentReplica: If a replica is deleted successfully, it is moved to this state. Valid previous state is - * ReplicaDeletionSuccessful - */ -class ZkReplicaStateMachine(config: KafkaConfig, - stateChangeLogger: StateChangeLogger, - controllerContext: ControllerContext, - zkClient: KafkaZkClient, - controllerBrokerRequestBatch: ControllerBrokerRequestBatch) - extends ReplicaStateMachine(controllerContext) with Logging { - - private val controllerId = config.brokerId - this.logIdent = s"[ReplicaStateMachine controllerId=$controllerId] " - - override def handleStateChanges(replicas: Seq[PartitionAndReplica], targetState: ReplicaState): Unit = { - throw new UnsupportedOperationException() - } -} - -sealed trait ReplicaState { - def state: Byte - def validPreviousStates: Set[ReplicaState] -} - -case object NewReplica extends ReplicaState { - val state: Byte = 1 - val validPreviousStates: Set[ReplicaState] = Set(NonExistentReplica) -} - -case object OnlineReplica extends ReplicaState { - val state: Byte = 2 - val validPreviousStates: Set[ReplicaState] = Set(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionIneligible) -} - -case object OfflineReplica extends ReplicaState { - val state: Byte = 3 - val validPreviousStates: Set[ReplicaState] = Set(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionIneligible) -} - -case object ReplicaDeletionStarted extends ReplicaState { - val state: Byte = 4 - val validPreviousStates: Set[ReplicaState] = Set(OfflineReplica) -} - -case object ReplicaDeletionSuccessful extends ReplicaState { - val state: Byte = 5 - val validPreviousStates: Set[ReplicaState] = Set(ReplicaDeletionStarted) -} - -case object ReplicaDeletionIneligible extends ReplicaState { - val state: Byte = 6 - val validPreviousStates: Set[ReplicaState] = Set(OfflineReplica, ReplicaDeletionStarted) -} - -case object NonExistentReplica extends ReplicaState { - val state: Byte = 7 - val validPreviousStates: Set[ReplicaState] = Set(ReplicaDeletionSuccessful) -} diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala deleted file mode 100755 index f36338edaae9a..0000000000000 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ /dev/null @@ -1,357 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.controller - -import kafka.server.KafkaConfig -import kafka.utils.Logging -import kafka.zk.KafkaZkClient -import org.apache.kafka.common.TopicPartition - -import scala.collection.Set -import scala.collection.mutable - -trait DeletionClient { - def deleteTopic(topic: String, epochZkVersion: Int): Unit - def deleteTopicDeletions(topics: Seq[String], epochZkVersion: Int): Unit - def mutePartitionModifications(topic: String): Unit - def sendMetadataUpdate(partitions: Set[TopicPartition]): Unit -} - -class ControllerDeletionClient(controller: KafkaController, zkClient: KafkaZkClient) extends DeletionClient { - override def deleteTopic(topic: String, epochZkVersion: Int): Unit = { - zkClient.deleteTopicZNode(topic, epochZkVersion) - zkClient.deleteTopicConfigs(Seq(topic), epochZkVersion) - zkClient.deleteTopicDeletions(Seq(topic), epochZkVersion) - } - - override def deleteTopicDeletions(topics: Seq[String], epochZkVersion: Int): Unit = { - zkClient.deleteTopicDeletions(topics, epochZkVersion) - } - - override def mutePartitionModifications(topic: String): Unit = { - controller.unregisterPartitionModificationsHandlers(Seq(topic)) - } - - override def sendMetadataUpdate(partitions: Set[TopicPartition]): Unit = { - controller.sendUpdateMetadataRequest(controller.controllerContext.liveOrShuttingDownBrokerIds.toSeq, partitions) - } -} - -/** - * This manages the state machine for topic deletion. - * 1. TopicCommand issues topic deletion by creating a new admin path /admin/delete_topics/ - * 2. The controller listens for child changes on /admin/delete_topic and starts topic deletion for the respective topics - * 3. The controller's ControllerEventThread handles topic deletion. A topic will be ineligible - * for deletion in the following scenarios - - * 3.1 broker hosting one of the replicas for that topic goes down - * 3.2 partition reassignment for partitions of that topic is in progress - * 4. Topic deletion is resumed when - - * 4.1 broker hosting one of the replicas for that topic is started - * 4.2 partition reassignment for partitions of that topic completes - * 5. Every replica for a topic being deleted is in either of the 3 states - - * 5.1 TopicDeletionStarted Replica enters TopicDeletionStarted phase when onPartitionDeletion is invoked. - * This happens when the child change watch for /admin/delete_topics fires on the controller. As part of this state - * change, the controller sends StopReplicaRequests to all replicas. It registers a callback for the - * StopReplicaResponse when deletePartition=true thereby invoking a callback when a response for delete replica - * is received from every replica) - * 5.2 TopicDeletionSuccessful moves replicas from - * TopicDeletionStarted->TopicDeletionSuccessful depending on the error codes in StopReplicaResponse - * 5.3 TopicDeletionFailed moves replicas from - * TopicDeletionStarted->TopicDeletionFailed depending on the error codes in StopReplicaResponse. - * In general, if a broker dies and if it hosted replicas for topics being deleted, the controller marks the - * respective replicas in TopicDeletionFailed state in the onBrokerFailure callback. The reason is that if a - * broker fails before the request is sent and after the replica is in TopicDeletionStarted state, - * it is possible that the replica will mistakenly remain in TopicDeletionStarted state and topic deletion - * will not be retried when the broker comes back up. - * 6. A topic is marked successfully deleted only if all replicas are in TopicDeletionSuccessful - * state. Topic deletion teardown mode deletes all topic state from the controllerContext - * as well as from zookeeper. This is the only time the /brokers/topics/ path gets deleted. On the other hand, - * if no replica is in TopicDeletionStarted state and at least one replica is in TopicDeletionFailed state, then - * it marks the topic for deletion retry. - */ -class TopicDeletionManager(config: KafkaConfig, - controllerContext: ControllerContext, - replicaStateMachine: ReplicaStateMachine, - partitionStateMachine: PartitionStateMachine, - client: DeletionClient) extends Logging { - this.logIdent = s"[Topic Deletion Manager ${config.brokerId}] " - val isDeleteTopicEnabled: Boolean = config.deleteTopicEnable - - def init(initialTopicsToBeDeleted: Set[String], initialTopicsIneligibleForDeletion: Set[String]): Unit = { - info(s"Initializing manager with initial deletions: $initialTopicsToBeDeleted, " + - s"initial ineligible deletions: $initialTopicsIneligibleForDeletion") - - if (isDeleteTopicEnabled) { - controllerContext.queueTopicDeletion(initialTopicsToBeDeleted) - controllerContext.topicsIneligibleForDeletion ++= initialTopicsIneligibleForDeletion & controllerContext.topicsToBeDeleted - } else { - // if delete topic is disabled clean the topic entries under /admin/delete_topics - info(s"Removing $initialTopicsToBeDeleted since delete topic is disabled") - client.deleteTopicDeletions(initialTopicsToBeDeleted.toSeq, controllerContext.epochZkVersion) - } - } - - def tryTopicDeletion(): Unit = { - if (isDeleteTopicEnabled) { - resumeDeletions() - } - } - - /** - * Invoked by the child change listener on /admin/delete_topics to queue up the topics for deletion. The topic gets added - * to the topicsToBeDeleted list and only gets removed from the list when the topic deletion has completed successfully - * i.e. all replicas of all partitions of that topic are deleted successfully. - * @param topics Topics that should be deleted - */ - def enqueueTopicsForDeletion(topics: Set[String]): Unit = { - if (isDeleteTopicEnabled) { - controllerContext.queueTopicDeletion(topics) - resumeDeletions() - } - } - - /** - * Invoked when any event that can possibly resume topic deletion occurs. These events include - - * 1. New broker starts up. Any replicas belonging to topics queued up for deletion can be deleted since the broker is up - * 2. Partition reassignment completes. Any partitions belonging to topics queued up for deletion finished reassignment - * @param topics Topics for which deletion can be resumed - */ - def resumeDeletionForTopics(topics: Set[String] = Set.empty): Unit = { - if (isDeleteTopicEnabled) { - val topicsToResumeDeletion = topics & controllerContext.topicsToBeDeleted - if (topicsToResumeDeletion.nonEmpty) { - controllerContext.topicsIneligibleForDeletion --= topicsToResumeDeletion - resumeDeletions() - } - } - } - - /** - * Invoked when a broker that hosts replicas for topics to be deleted goes down. Also invoked when the callback for - * StopReplicaResponse receives an error code for the replicas of a topic to be deleted. As part of this, the replicas - * are moved from ReplicaDeletionStarted to ReplicaDeletionIneligible state. Also, the topic is added to the list of topics - * ineligible for deletion until further notice. - * @param replicas Replicas for which deletion has failed - */ - def failReplicaDeletion(replicas: Set[PartitionAndReplica]): Unit = { - if (isDeleteTopicEnabled) { - val replicasThatFailedToDelete = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) - if (replicasThatFailedToDelete.nonEmpty) { - val topics = replicasThatFailedToDelete.map(_.topic) - debug(s"Deletion failed for replicas ${replicasThatFailedToDelete.mkString(",")}. Halting deletion for topics $topics") - replicaStateMachine.handleStateChanges(replicasThatFailedToDelete.toSeq, ReplicaDeletionIneligible) - markTopicIneligibleForDeletion(topics, reason = "replica deletion failure") - resumeDeletions() - } - } - } - - /** - * Halt delete topic if - - * 1. replicas being down - * 2. partition reassignment in progress for some partitions of the topic - * @param topics Topics that should be marked ineligible for deletion. No op if the topic is was not previously queued up for deletion - */ - def markTopicIneligibleForDeletion(topics: Set[String], reason: => String): Unit = { - if (isDeleteTopicEnabled) { - val newTopicsToHaltDeletion = controllerContext.topicsToBeDeleted & topics - controllerContext.topicsIneligibleForDeletion ++= newTopicsToHaltDeletion - if (newTopicsToHaltDeletion.nonEmpty) - info(s"Halted deletion of topics ${newTopicsToHaltDeletion.mkString(",")} due to $reason") - } - } - - private def isTopicIneligibleForDeletion(topic: String): Boolean = { - if (isDeleteTopicEnabled) { - controllerContext.topicsIneligibleForDeletion.contains(topic) - } else - true - } - - private def isTopicDeletionInProgress(topic: String): Boolean = { - if (isDeleteTopicEnabled) { - controllerContext.isAnyReplicaInState(topic, ReplicaDeletionStarted) - } else - false - } - - def isTopicQueuedUpForDeletion(topic: String): Boolean = { - if (isDeleteTopicEnabled) { - controllerContext.isTopicQueuedUpForDeletion(topic) - } else - false - } - - /** - * Invoked by the StopReplicaResponse callback when it receives no error code for a replica of a topic to be deleted. - * As part of this, the replicas are moved from ReplicaDeletionStarted to ReplicaDeletionSuccessful state. Tears down - * the topic if all replicas of a topic have been successfully deleted - * @param replicas Replicas that were successfully deleted by the broker - */ - def completeReplicaDeletion(replicas: Set[PartitionAndReplica]): Unit = { - val successfullyDeletedReplicas = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) - debug(s"Deletion successfully completed for replicas ${successfullyDeletedReplicas.mkString(",")}") - replicaStateMachine.handleStateChanges(successfullyDeletedReplicas.toSeq, ReplicaDeletionSuccessful) - resumeDeletions() - } - - /** - * Topic deletion can be retried if - - * 1. Topic deletion is not already complete - * 2. Topic deletion is currently not in progress for that topic - * 3. Topic is currently not marked ineligible for deletion - * @param topic Topic - * @return Whether or not deletion can be retried for the topic - */ - private def isTopicEligibleForDeletion(topic: String): Boolean = { - controllerContext.isTopicQueuedUpForDeletion(topic) && - !isTopicDeletionInProgress(topic) && - !isTopicIneligibleForDeletion(topic) - } - - /** - * If the topic is queued for deletion but deletion is not currently under progress, then deletion is retried for that topic - * To ensure a successful retry, reset states for respective replicas from ReplicaDeletionIneligible to OfflineReplica state - * @param topics Topics for which deletion should be retried - */ - private def retryDeletionForIneligibleReplicas(topics: Set[String]): Unit = { - // reset replica states from ReplicaDeletionIneligible to OfflineReplica - val failedReplicas = topics.flatMap(controllerContext.replicasInState(_, ReplicaDeletionIneligible)) - debug(s"Retrying deletion of topics ${topics.mkString(",")} since replicas ${failedReplicas.mkString(",")} were not successfully deleted") - replicaStateMachine.handleStateChanges(failedReplicas.toSeq, OfflineReplica) - } - - private def completeDeleteTopic(topic: String): Unit = { - // deregister partition change listener on the deleted topic. This is to prevent the partition change listener - // firing before the new topic listener when a deleted topic gets auto created - client.mutePartitionModifications(topic) - val replicasForDeletedTopic = controllerContext.replicasInState(topic, ReplicaDeletionSuccessful) - // controller will remove this replica from the state machine as well as its partition assignment cache - replicaStateMachine.handleStateChanges(replicasForDeletedTopic.toSeq, NonExistentReplica) - client.deleteTopic(topic, controllerContext.epochZkVersion) - controllerContext.removeTopic(topic) - } - - /** - * Invoked with the list of topics to be deleted - * It invokes onPartitionDeletion for all partitions of a topic. - * The updateMetadataRequest is also going to set the leader for the topics being deleted to - * [[org.apache.kafka.metadata.LeaderAndIsr#LeaderDuringDelete]]. This lets each broker know that this topic is being deleted and can be - * removed from their caches. - */ - private def onTopicDeletion(topics: Set[String]): Unit = { - val unseenTopicsForDeletion = topics.diff(controllerContext.topicsWithDeletionStarted) - if (unseenTopicsForDeletion.nonEmpty) { - val unseenPartitionsForDeletion = unseenTopicsForDeletion.flatMap(controllerContext.partitionsForTopic) - partitionStateMachine.handleStateChanges(unseenPartitionsForDeletion.toSeq, OfflinePartition) - partitionStateMachine.handleStateChanges(unseenPartitionsForDeletion.toSeq, NonExistentPartition) - // adding of unseenTopicsForDeletion to topics with deletion started must be done after the partition - // state changes to make sure the offlinePartitionCount metric is properly updated - controllerContext.beginTopicDeletion(unseenTopicsForDeletion) - } - - // send update metadata so that brokers stop serving data for topics to be deleted - client.sendMetadataUpdate(topics.flatMap(controllerContext.partitionsForTopic)) - - onPartitionDeletion(topics) - } - - /** - * Invoked by onTopicDeletion with the list of partitions for topics to be deleted - * It does the following - - * 1. Move all dead replicas directly to ReplicaDeletionIneligible state. Also mark the respective topics ineligible - * for deletion if some replicas are dead since it won't complete successfully anyway - * 2. Move all replicas for the partitions to OfflineReplica state. This will send StopReplicaRequest to the replicas - * and LeaderAndIsrRequest to the leader with the shrunk ISR. When the leader replica itself is moved to OfflineReplica state, - * it will skip sending the LeaderAndIsrRequest since the leader will be updated to -1 - * 3. Move all replicas to ReplicaDeletionStarted state. This will send StopReplicaRequest with deletePartition=true. And - * will delete all persistent data from all replicas of the respective partitions - */ - private def onPartitionDeletion(topicsToBeDeleted: Set[String]): Unit = { - val allDeadReplicas = mutable.ListBuffer.empty[PartitionAndReplica] - val allReplicasForDeletionRetry = mutable.ListBuffer.empty[PartitionAndReplica] - val allTopicsIneligibleForDeletion = mutable.Set.empty[String] - - topicsToBeDeleted.foreach { topic => - val (aliveReplicas, deadReplicas) = controllerContext.replicasForTopic(topic).partition { r => - controllerContext.isReplicaOnline(r.replica, r.topicPartition) - } - - val successfullyDeletedReplicas = controllerContext.replicasInState(topic, ReplicaDeletionSuccessful) - val replicasForDeletionRetry = aliveReplicas.diff(successfullyDeletedReplicas) - - allDeadReplicas ++= deadReplicas - allReplicasForDeletionRetry ++= replicasForDeletionRetry - - if (deadReplicas.nonEmpty) { - debug(s"Dead Replicas (${deadReplicas.mkString(",")}) found for topic $topic") - allTopicsIneligibleForDeletion += topic - } - } - - // move dead replicas directly to failed state - replicaStateMachine.handleStateChanges(allDeadReplicas, ReplicaDeletionIneligible) - // send stop replica to all followers that are not in the OfflineReplica state so they stop sending fetch requests to the leader - replicaStateMachine.handleStateChanges(allReplicasForDeletionRetry, OfflineReplica) - replicaStateMachine.handleStateChanges(allReplicasForDeletionRetry, ReplicaDeletionStarted) - - if (allTopicsIneligibleForDeletion.nonEmpty) { - markTopicIneligibleForDeletion(allTopicsIneligibleForDeletion, reason = "offline replicas") - } - } - - private def resumeDeletions(): Unit = { - val topicsQueuedForDeletion = Set.empty[String] ++ controllerContext.topicsToBeDeleted - val topicsEligibleForRetry = mutable.Set.empty[String] - val topicsEligibleForDeletion = mutable.Set.empty[String] - - if (topicsQueuedForDeletion.nonEmpty) - info(s"Handling deletion for topics ${topicsQueuedForDeletion.mkString(",")}") - - topicsQueuedForDeletion.foreach { topic => - // if all replicas are marked as deleted successfully, then topic deletion is done - if (controllerContext.areAllReplicasInState(topic, ReplicaDeletionSuccessful)) { - // clear up all state for this topic from controller cache and zookeeper - completeDeleteTopic(topic) - info(s"Deletion of topic $topic successfully completed") - } else if (!controllerContext.isAnyReplicaInState(topic, ReplicaDeletionStarted)) { - // if you come here, then no replica is in TopicDeletionStarted and all replicas are not in - // TopicDeletionSuccessful. That means, that either given topic haven't initiated deletion - // or there is at least one failed replica (which means topic deletion should be retried). - if (controllerContext.isAnyReplicaInState(topic, ReplicaDeletionIneligible)) { - topicsEligibleForRetry += topic - } - } - - // Add topic to the eligible set if it is eligible for deletion. - if (isTopicEligibleForDeletion(topic)) { - info(s"Deletion of topic $topic (re)started") - topicsEligibleForDeletion += topic - } - } - - // topic deletion retry will be kicked off - if (topicsEligibleForRetry.nonEmpty) { - retryDeletionForIneligibleReplicas(topicsEligibleForRetry) - } - - // topic deletion will be kicked off - if (topicsEligibleForDeletion.nonEmpty) { - onTopicDeletion(topicsEligibleForDeletion) - } - } -} diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index effd713579910..d42d83fde4d72 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -948,8 +948,7 @@ class LogManager(logDirs: Seq[File], def updateTopicConfig(topic: String, newTopicConfig: Properties, isRemoteLogStorageSystemEnabled: Boolean, - wasRemoteLogEnabled: Boolean, - fromZK: Boolean): Unit = { + wasRemoteLogEnabled: Boolean): Unit = { topicConfigUpdated(topic) val logs = logsByTopic(topic) // Combine the default properties with the overrides in zk to create the new LogConfig @@ -959,10 +958,6 @@ class LogManager(logDirs: Seq[File], // Otherwise we risk someone creating a tiered-topic, disabling Tiered Storage cluster-wide and the check // failing since the logs for the topic are non-existent. LogConfig.validateRemoteStorageOnlyIfSystemEnabled(newLogConfig.values(), isRemoteLogStorageSystemEnabled, true) - // `remote.log.delete.on.disable` and `remote.log.copy.disable` are unsupported in ZK mode - if (fromZK) { - LogConfig.validateNoInvalidRemoteStorageConfigsInZK(newLogConfig.values()) - } LogConfig.validateTurningOffRemoteStorageWithDelete(newLogConfig.values(), wasRemoteLogEnabled, isRemoteLogStorageEnabled) LogConfig.validateRetentionConfigsWhenRemoteCopyDisabled(newLogConfig.values(), isRemoteLogStorageEnabled) if (logs.nonEmpty) { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 22a135d1b4592..47b89dd1de18f 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -387,7 +387,7 @@ class BrokerServer( transactionCoordinator, shareCoordinator) dynamicConfigHandlers = Map[String, ConfigHandler]( - ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers, None), + ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers), ConfigType.BROKER -> new BrokerConfigHandler(config, quotaManagers), ConfigType.CLIENT_METRICS -> new ClientMetricsConfigHandler(clientMetricsManager), ConfigType.GROUP -> new GroupConfigHandler(groupCoordinator)) diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index e1bc4f9b393ac..eabe12d0d942a 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -19,24 +19,21 @@ package kafka.server import java.net.{InetAddress, UnknownHostException} import java.util.{Collections, Properties} -import kafka.controller.KafkaController import kafka.log.UnifiedLog import kafka.network.ConnectionQuotas import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.Logging -import org.apache.kafka.server.config.{QuotaConfig, ReplicationConfigs, ZooKeeperInternals} +import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.metrics.Quota._ import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.coordinator.group.GroupCoordinator -import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.common.StopPartition import org.apache.kafka.storage.internals.log.{LogStartOffsetIncrementReason, ThrottledReplicaListValidator} import scala.jdk.CollectionConverters._ import scala.collection.Seq -import scala.util.Try /** * The ConfigHandler is used to process broker configuration change notifications. @@ -51,8 +48,7 @@ trait ConfigHandler { */ class TopicConfigHandler(private val replicaManager: ReplicaManager, kafkaConfig: KafkaConfig, - val quotas: QuotaManagers, - kafkaController: Option[KafkaController]) extends ConfigHandler with Logging { + val quotas: QuotaManagers) extends ConfigHandler with Logging { private def updateLogConfig(topic: String, topicConfig: Properties): Unit = { @@ -62,9 +58,8 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, val wasRemoteLogEnabled = logs.exists(_.remoteLogEnabled()) val wasCopyDisabled = logs.exists(_.config.remoteLogCopyDisable()) - // kafkaController is only defined in Zookeeper's mode logManager.updateTopicConfig(topic, topicConfig, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), - wasRemoteLogEnabled, kafkaController.isDefined) + wasRemoteLogEnabled) maybeUpdateRemoteLogComponents(topic, logs, wasRemoteLogEnabled, wasCopyDisabled) } @@ -130,10 +125,6 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, } updateThrottledList(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, quotas.leader) updateThrottledList(QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, quotas.follower) - - if (Try(topicConfig.getProperty(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG).toBoolean).getOrElse(false)) { - kafkaController.foreach(_.enableTopicUncleanLeaderElection(topic)) - } } def parseThrottledPartitions(topicConfig: Properties, brokerId: Int, prop: String): Seq[Int] = { @@ -187,37 +178,6 @@ class QuotaConfigHandler(private val quotaManagers: QuotaManagers) { } } -/** - * The ClientIdConfigHandler will process clientId config changes in ZK. - * The callback provides the clientId and the full properties set read from ZK. - */ -class ClientIdConfigHandler(private val quotaManagers: QuotaManagers) extends QuotaConfigHandler(quotaManagers) with ConfigHandler { - - def processConfigChanges(sanitizedClientId: String, clientConfig: Properties): Unit = { - updateQuotaConfig(None, Some(sanitizedClientId), clientConfig) - } -} - -/** - * The UserConfigHandler will process and quota changes in ZK. - * The callback provides the node name containing sanitized user principal, sanitized client-id if this is - * a update and the full properties set read from ZK. - */ -class UserConfigHandler(private val quotaManagers: QuotaManagers, val credentialProvider: CredentialProvider) extends QuotaConfigHandler(quotaManagers) with ConfigHandler { - - def processConfigChanges(quotaEntityPath: String, config: Properties): Unit = { - // Entity path is or /clients/ - val entities = quotaEntityPath.split("/") - if (entities.length != 1 && entities.length != 3) - throw new IllegalArgumentException("Invalid quota entity path: " + quotaEntityPath) - val sanitizedUser = entities(0) - val sanitizedClientId = if (entities.length == 3) Some(entities(2)) else None - updateQuotaConfig(Some(sanitizedUser), sanitizedClientId, config) - if (sanitizedClientId.isEmpty && sanitizedUser != ZooKeeperInternals.DEFAULT_STRING) - credentialProvider.updateCredentials(Sanitizer.desanitize(sanitizedUser), config) - } -} - class IpConfigHandler(private val connectionQuotas: ConnectionQuotas) extends ConfigHandler with Logging { def processConfigChanges(ip: String, config: Properties): Unit = { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 6d792c7647a4a..0428fc40380f5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -18,7 +18,7 @@ package kafka.server import com.yammer.metrics.core.Meter import kafka.cluster.{Partition, PartitionListener} -import kafka.controller.{KafkaController, StateChangeLogger} +import kafka.controller.StateChangeLogger import kafka.log.remote.RemoteLogManager import kafka.log.{LogManager, UnifiedLog} import kafka.server.HostedPartition.Online @@ -49,7 +49,7 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.{Exit, Time} -import org.apache.kafka.common.{ElectionType, IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.common.{IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderAndIsr import org.apache.kafka.metadata.LeaderConstants.NO_LEADER @@ -317,7 +317,7 @@ class ReplicaManager(val config: KafkaConfig, config.shareGroupConfig.shareFetchPurgatoryPurgeIntervalRequests)) /* epoch of the controller that last changed the leader */ - @volatile private[server] var controllerEpoch: Int = KafkaController.InitialControllerEpoch + @volatile private[server] var controllerEpoch: Int = 0 protected val localBrokerId = config.brokerId protected val allPartitions = new Pool[TopicPartition, HostedPartition]( valueFactory = Some(tp => HostedPartition.Online(Partition(tp, time, this))) @@ -2739,47 +2739,6 @@ class ReplicaManager(val config: KafkaConfig, } } - def electLeaders( - controller: KafkaController, - partitions: Set[TopicPartition], - electionType: ElectionType, - responseCallback: Map[TopicPartition, ApiError] => Unit, - requestTimeout: Int - ): Unit = { - - val deadline = time.milliseconds() + requestTimeout - - def electionCallback(results: Map[TopicPartition, Either[ApiError, Int]]): Unit = { - val expectedLeaders = mutable.Map.empty[TopicPartition, Int] - val failures = mutable.Map.empty[TopicPartition, ApiError] - results.foreach { - case (partition, Right(leader)) => expectedLeaders += partition -> leader - case (partition, Left(error)) => failures += partition -> error - } - if (expectedLeaders.nonEmpty) { - val watchKeys = expectedLeaders.iterator.map { - case (tp, _) => new TopicPartitionOperationKey(tp) - }.toList.asJava - - delayedElectLeaderPurgatory.tryCompleteElseWatch( - new DelayedElectLeader( - math.max(0, deadline - time.milliseconds()), - expectedLeaders, - failures, - this, - responseCallback - ), - watchKeys - ) - } else { - // There are no partitions actually being elected, so return immediately - responseCallback(failures) - } - } - - controller.electLeaders(partitions, electionType, electionCallback) - } - def activeProducerState(requestPartition: TopicPartition): DescribeProducersResponseData.PartitionResponse = { getPartitionOrError(requestPartition) match { case Left(error) => new DescribeProducersResponseData.PartitionResponse() diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index e03818f0be34f..8b62d3c178292 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -18,14 +18,10 @@ package kafka.zk import java.util.Properties import kafka.cluster.Broker -import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} +import kafka.controller.ReplicaAssignment import kafka.utils.Logging -import kafka.zk.TopicZNode.TopicIdReplicaAssignment -import kafka.zookeeper._ import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.{TopicPartition, Uuid} -import org.apache.kafka.metadata.LeaderAndIsr -import org.apache.kafka.storage.internals.log.LogConfig import scala.collection.{Map, Seq} @@ -38,119 +34,6 @@ import scala.collection.{Map, Seq} * monolithic [[kafka.zk.ZkData]] is the way to go. */ class KafkaZkClient() extends AutoCloseable with Logging { - import KafkaZkClient._ - - /** - * 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. - * - * @param path the path to create (with the monotonically increasing number appended) - * @param data the znode data - * @return the created path (including the appended monotonically increasing number) - */ - private[kafka] def createSequentialPersistentPath(path: String, data: Array[Byte]): String = { - throw new UnsupportedOperationException() - } - - /** - * Registers the broker in zookeeper and return the broker epoch. - * @param brokerInfo payload of the broker znode - * @return broker epoch (znode create transaction id) - */ - def registerBroker(brokerInfo: BrokerInfo): Long = { - 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 - */ - def registerControllerAndIncrementControllerEpoch(controllerId: Int): (Int, Int) = { - throw new UnsupportedOperationException() - } - - def updateBrokerInfo(brokerInfo: BrokerInfo): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Gets topic partition states for the given partitions. - * @param partitions the partitions for which we want to get states. - * @return sequence of GetDataResponses whose contexts are the partitions they are associated with. - */ - def getTopicPartitionStatesRaw(partitions: Seq[TopicPartition]): Seq[GetDataResponse] = { - throw new UnsupportedOperationException() - } - - /** - * Sets topic partition states for the given partitions. - * @param leaderIsrAndControllerEpochs the partition states of each partition whose state we wish to set. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - * @return sequence of SetDataResponse whose contexts are the partitions they are associated with. - */ - def setTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[SetDataResponse] = { - throw new UnsupportedOperationException() - } - - /** - * Creates topic partition state znodes for the given partitions. - * @param leaderIsrAndControllerEpochs the partition states of each partition whose state we wish to set. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - * @return sequence of CreateResponse whose contexts are the partitions they are associated with. - */ - def createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch], expectedControllerEpochZkVersion: Int): Seq[CreateResponse] = { - throw new UnsupportedOperationException() - } - - /** - * Sets the controller epoch conditioned on the given epochZkVersion. - * @param epoch the epoch to set - * @param epochZkVersion the expected version number of the epoch znode. - * @return SetDataResponse - */ - def setControllerEpochRaw(epoch: Int, epochZkVersion: Int): SetDataResponse = { - throw new UnsupportedOperationException() - } - - /** - * Creates the controller epoch znode. - * @param epoch the epoch to set - * @return CreateResponse - */ - def createControllerEpochRaw(epoch: Int): CreateResponse = { - throw new UnsupportedOperationException() - } - - /** - * Update the partition states of multiple partitions in zookeeper. - * @param leaderAndIsrs The partition states to update. - * @param controllerEpoch The current controller epoch. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - * @return UpdateLeaderAndIsrResult instance containing per partition results. - */ - def updateLeaderAndIsr( - leaderAndIsrs: Map[TopicPartition, LeaderAndIsr], - controllerEpoch: Int, - expectedControllerEpochZkVersion: Int - ): UpdateLeaderAndIsrResult = { - throw new UnsupportedOperationException() - } - - /** - * Get log configs that merge local configs with topic-level configs in zookeeper. - * @param topics The topics to get log configs for. - * @param config The local configs. - * @return A tuple of two values: - * 1. The successfully gathered log configs - * 2. Exceptions corresponding to failed log config lookups. - */ - def getLogConfigs( - topics: Set[String], - config: java.util.Map[String, AnyRef] - ): (Map[String, LogConfig], Map[String, Exception]) = { - throw new UnsupportedOperationException() - } /** * Get entity configs for a given entity name @@ -162,10 +45,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - def getEntitiesConfigs(rootEntityType: String, sanitizedEntityNames: Set[String]): Map[String, Properties] = { - throw new UnsupportedOperationException() - } - /** * Sets or creates the entity znode path with the given configs depending * on whether it already exists or not. @@ -217,29 +96,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Gets all brokers with broker epoch in the cluster. - * @return map of broker to epoch in the cluster. - */ - def getAllBrokerAndEpochsInCluster: Map[Broker, Long] = { - throw new UnsupportedOperationException() - } - - /** - * Get a broker from ZK - * @return an optional Broker - */ - def getBroker(brokerId: Int): Option[Broker] = { - throw new UnsupportedOperationException() - } - - /** - * Gets the list of sorted broker Ids - */ - def getSortedBrokerList: Seq[Int] = { - throw new UnsupportedOperationException() - } - /** * Gets all topics in the cluster. * @param registerWatch indicates if a watch must be registered or not @@ -258,31 +114,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Adds a topic ID to existing topic and replica assignments - * @param topicIdReplicaAssignments the TopicIDReplicaAssignments to add a topic ID to - * @return the updated TopicIdReplicaAssignments including the newly created topic IDs - */ - def setTopicIds(topicIdReplicaAssignments: collection.Set[TopicIdReplicaAssignment], - expectedControllerEpochZkVersion: Int): Set[TopicIdReplicaAssignment] = { - throw new UnsupportedOperationException() - } - - /** - * Sets the topic znode with the given assignment. - * @param topic the topic whose assignment is being set. - * @param topicId unique topic ID for the topic if the version supports it - * @param assignment the partition to replica mapping to set for the given topic - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - * @return SetDataResponse - */ - def setTopicAssignmentRaw(topic: String, - topicId: Option[Uuid], - assignment: collection.Map[TopicPartition, ReplicaAssignment], - expectedControllerEpochZkVersion: Int): SetDataResponse = { - throw new UnsupportedOperationException() - } - /** * Sets the topic znode with the given assignment. * @param topic the topic whose assignment is being set. @@ -309,40 +140,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Gets the log dir event notifications as strings. These strings are the znode names and not the absolute znode path. - * @return sequence of znode names and not the absolute znode path. - */ - def getAllLogDirEventNotifications: Seq[String] = { - throw new UnsupportedOperationException() - } - - /** - * Reads each of the log dir event notifications associated with the given sequence numbers and extracts the broker ids. - * @param sequenceNumbers the sequence numbers associated with the log dir event notifications. - * @return broker ids associated with the given log dir event notifications. - */ - def getBrokerIdsFromLogDirEvents(sequenceNumbers: Seq[String]): Seq[Int] = { - throw new UnsupportedOperationException() - } - - /** - * Deletes all log dir event notifications. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteLogDirEventNotifications(expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Deletes the log dir event notifications associated with the given sequence numbers. - * @param sequenceNumbers the sequence numbers associated with the log dir event notifications to be deleted. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteLogDirEventNotifications(sequenceNumbers: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - /** * Gets the topic IDs for the given topics. * @param topics the topics we wish to retrieve the Topic IDs for @@ -362,15 +159,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Gets the TopicID and replica assignments for the given topics. - * @param topics the topics whose partitions we wish to get the assignments for. - * @return the TopicIdReplicaAssignment for each partition for the given topics. - */ - def getReplicaAssignmentAndTopicIdForTopics(topics: Set[String]): Set[TopicIdReplicaAssignment] = { - throw new UnsupportedOperationException() - } - /** * Gets the replica assignments for the given topics. * @param topics the topics whose partitions we wish to get the assignments for. @@ -380,24 +168,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Gets partition the assignments for the given topics. - * @param topics the topics whose partitions we wish to get the assignments for. - * @return the partition assignment for each partition from the given topics. - */ - def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, ReplicaAssignment]] = { - throw new UnsupportedOperationException() - } - - /** - * Gets the partition numbers for the given topics - * @param topics the topics whose partitions we wish to get. - * @return the partition array for each topic from the given topics. - */ - def getPartitionsForTopics(topics: Set[String]): Map[String, Seq[Int]] = { - throw new UnsupportedOperationException() - } - /** * Gets the partition count for a given topic * @param topic The topic to get partition count for. @@ -407,15 +177,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Gets the assigned replicas for a specific topic and partition - * @param topicPartition TopicAndPartition to get assigned replicas for . - * @return List of assigned replicas - */ - def getReplicasForPartition(topicPartition: TopicPartition): Seq[Int] = { - throw new UnsupportedOperationException() - } - /** * Gets all partitions in the cluster * @return all partitions in the cluster @@ -424,28 +185,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Gets the data and version at the given zk path - * @param path zk node path - * @return A tuple of 2 elements, where first element is zk node data as an array of bytes - * and second element is zk node version. - * returns (None, ZkVersion.UnknownVersion) if node doesn't exist and throws exception for any error - */ - def getDataAndVersion(path: String): (Option[Array[Byte]], Int) = { - throw new UnsupportedOperationException() - } - - /** - * Gets the data and Stat at the given zk path - * @param path zk node path - * @return A tuple of 2 elements, where first element is zk node data as an array of bytes - * and second element is zk node stats. - * returns (None, ZkStat.NoStat) if node doesn't exists and throws exception for any error - */ - def getDataAndStat(path: String): (Option[Array[Byte]], Stat) = { - throw new UnsupportedOperationException() - } - /** * Gets all the child nodes at a given zk node path * @param path the path to check @@ -455,19 +194,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't - * exist, the current version is not the expected version, etc.) return (false, ZkVersion.UnknownVersion) - * - * When there is a ConnectionLossException during the conditional update, ZookeeperClient will retry the update and may fail - * since the previous update may have succeeded (but the stored zkVersion no longer matches the expected one). - * In this case, we will run the optionalChecker to further check if the previous write did indeed succeeded. - */ - def conditionalUpdatePath(path: String, data: Array[Byte], expectVersion: Int, - optionalChecker: Option[(KafkaZkClient, String, Array[Byte]) => (Boolean,Int)] = None): (Boolean, Int) = { - throw new UnsupportedOperationException() - } - /** * Creates the delete topic znode. * @param topicName topic name @@ -486,31 +212,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Get all topics marked for deletion. - * @return sequence of topics marked for deletion. - */ - def getTopicDeletions: Seq[String] = { - throw new UnsupportedOperationException() - } - - /** - * Remove the given topics from the topics marked for deletion. - * @param topics the topics to remove. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteTopicDeletions(topics: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Returns all reassignments. - * @return the reassignments for each partition. - */ - def getPartitionReassignment: collection.Map[TopicPartition, Seq[Int]] = { - throw new UnsupportedOperationException() - } - /** * Sets or creates the partition reassignment znode with the given reassignment depending on whether it already * exists or not. @@ -525,134 +226,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Creates the partition reassignment znode with the given reassignment. - * @param reassignment the reassignment to set on the reassignment znode. - * @throws KeeperException if there is an error while creating the znode. - */ - def createPartitionReassignment(reassignment: Map[TopicPartition, Seq[Int]]): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Deletes the partition reassignment znode. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deletePartitionReassignment(expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Checks if reassign partitions is in progress. - * @return true if reassign partitions is in progress, else false. - */ - def reassignPartitionsInProgress: Boolean = { - throw new UnsupportedOperationException() - } - - /** - * Gets topic partition states for the given partitions. - * @param partitions the partitions for which we want to get states. - * @return map containing LeaderIsrAndControllerEpoch of each partition for we were able to lookup the partition state. - */ - def getTopicPartitionStates(partitions: Seq[TopicPartition]): Map[TopicPartition, LeaderIsrAndControllerEpoch] = { - throw new UnsupportedOperationException() - } - - /** - * Gets topic partition state for the given partition. - * @param partition the partition for which we want to get state. - * @return LeaderIsrAndControllerEpoch of the partition state if exists, else None - */ - def getTopicPartitionState(partition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { - throw new UnsupportedOperationException() - } - - /** - * Gets the leader for a given partition - * @param partition The partition for which we want to get leader. - * @return optional integer if the leader exists and None otherwise. - */ - def getLeaderForPartition(partition: TopicPartition): Option[Int] = - throw new UnsupportedOperationException() - - /** - * Gets the in-sync replicas (ISR) for a specific topicPartition - * @param partition The partition for which we want to get ISR. - * @return optional ISR if exists and None otherwise - */ - def getInSyncReplicasForPartition(partition: TopicPartition): Option[Seq[Int]] = - throw new UnsupportedOperationException() - - - /** - * Gets the leader epoch for a specific topicPartition - * @param partition The partition for which we want to get the leader epoch - * @return optional integer if the leader exists and None otherwise - */ - def getEpochForPartition(partition: TopicPartition): Option[Int] = { - throw new UnsupportedOperationException() - } - - /** - * Gets the isr change notifications as strings. These strings are the znode names and not the absolute znode path. - * @return sequence of znode names and not the absolute znode path. - */ - def getAllIsrChangeNotifications: Seq[String] = { - throw new UnsupportedOperationException() - } - - /** - * Reads each of the isr change notifications associated with the given sequence numbers and extracts the partitions. - * @param sequenceNumbers the sequence numbers associated with the isr change notifications. - * @return partitions associated with the given isr change notifications. - */ - def getPartitionsFromIsrChangeNotifications(sequenceNumbers: Seq[String]): Seq[TopicPartition] = { - throw new UnsupportedOperationException() - } - - /** - * Deletes all isr change notifications. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteIsrChangeNotifications(expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Deletes the isr change notifications associated with the given sequence numbers. - * @param sequenceNumbers the sequence numbers associated with the isr change notifications to be deleted. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteIsrChangeNotifications(sequenceNumbers: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Creates preferred replica election znode with partitions undergoing election - * @param partitions the set of partitions - * @throws KeeperException if there is an error while creating the znode - */ - def createPreferredReplicaElection(partitions: Set[TopicPartition]): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Gets the partitions marked for preferred replica election. - * @return sequence of partitions. - */ - def getPreferredReplicaElection: Set[TopicPartition] = { - throw new UnsupportedOperationException() - } - - /** - * Deletes the preferred replica election znode. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deletePreferredReplicaElection(expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - /** * Gets the controller id. * @return optional integer that is Some if the controller znode exists and can be parsed and None otherwise. @@ -661,44 +234,6 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * Deletes the controller znode. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteController(expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Gets the controller epoch. - * @return optional (Int, Stat) that is Some if the controller epoch path exists and None otherwise. - */ - def getControllerEpoch: Option[(Int, Stat)] = { - throw new UnsupportedOperationException() - } - - /** - * Recursively deletes the topic znode. - * @param topic the topic whose topic znode we wish to delete. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteTopicZNode(topic: String, expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Deletes the topic configs for the given topics. - * @param topics the topics whose configs we wish to delete. - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - */ - def deleteTopicConfigs(topics: Seq[String], expectedControllerEpochZkVersion: Int): Unit = { - throw new UnsupportedOperationException() - } - - def propagateLogDirEvent(brokerId: Int): Unit = { - throw new UnsupportedOperationException() - } - /** * Deletes the zk node recursively * @param path path to delete @@ -753,189 +288,10 @@ class KafkaZkClient() extends AutoCloseable with Logging { throw new UnsupportedOperationException() } - /** - * This registers a ZNodeChangeHandler and attempts to register a watcher with an ExistsRequest, which allows data - * watcher registrations on paths which might not even exist. - * - * @param zNodeChangeHandler - * @return `true` if the path exists or `false` if it does not - * @throws KeeperException if an error is returned by ZooKeeper - */ - def registerZNodeChangeHandlerAndCheckExistence(zNodeChangeHandler: ZNodeChangeHandler): Boolean = { - throw new UnsupportedOperationException() - } - - /** - * See ZooKeeperClient.registerZNodeChangeHandler - * @param zNodeChangeHandler - */ - def registerZNodeChangeHandler(zNodeChangeHandler: ZNodeChangeHandler): Unit = { - throw new UnsupportedOperationException() - } - - /** - * See ZooKeeperClient.unregisterZNodeChangeHandler - * @param path - */ - def unregisterZNodeChangeHandler(path: String): Unit = { - throw new UnsupportedOperationException() - } - - /** - * See ZooKeeperClient.registerZNodeChildChangeHandler - * @param zNodeChildChangeHandler - */ - def registerZNodeChildChangeHandler(zNodeChildChangeHandler: ZNodeChildChangeHandler): Unit = { - throw new UnsupportedOperationException() - } - - /** - * See ZooKeeperClient.unregisterZNodeChildChangeHandler - * @param path - */ - def unregisterZNodeChildChangeHandler(path: String): Unit = { - throw new UnsupportedOperationException() - } - - /** - * - * @param stateChangeHandler - */ - def registerStateChangeHandler(stateChangeHandler: StateChangeHandler): Unit = { - throw new UnsupportedOperationException() - } - - /** - * - * @param name - */ - def unregisterStateChangeHandler(name: String): Unit = { - throw new UnsupportedOperationException() - } - /** * Close the underlying ZooKeeperClient. */ def close(): Unit = { throw new UnsupportedOperationException() } - - /** - * Get the committed offset for a topic partition and group - * @param group the group we wish to get offset for - * @param topicPartition the topic partition we wish to get the offset for - * @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] = { - throw new UnsupportedOperationException() - } - - /** - * Set the committed offset for a topic partition and group - * @param group the group whose offset is being set - * @param topicPartition the topic partition whose offset is being set - * @param offset the offset value - */ - def setOrCreateConsumerOffset(group: String, topicPartition: TopicPartition, offset: Long): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Get the cluster id. - * @return optional cluster id in String. - */ - def getClusterId: Option[String] = { - throw new UnsupportedOperationException() - } - - /** - * Return the ACLs of the node of the given path - * @param path the given path for the node - * @return the ACL array of the given node. - */ - def getAcl(path: String): Seq[ACL] = { - throw new UnsupportedOperationException() - } - - /** - * sets the ACLs to the node of the given path - * @param path the given path for the node - * @param acl the given acl for the node - */ - def setAcl(path: String, acl: Seq[ACL]): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Create the cluster Id. If the cluster id already exists, return the current cluster id. - * @return cluster id - */ - def createOrGetClusterId(proposedClusterId: String): String = { - throw new UnsupportedOperationException() - } - - /** - * Generate a broker id by updating the broker sequence id path in ZK and return the version of the path. - * The version is incremented by one on every update starting from 1. - * @return sequence number as the broker id - */ - def generateBrokerSequenceId(): Int = { - throw new UnsupportedOperationException() - } - - /** - * Pre-create top level paths in ZK if needed. - */ - def createTopLevelPaths(): Unit = { - throw new UnsupportedOperationException() - } - - /** - * Make sure a persistent path exists in ZK. - * @param path - */ - def makeSurePersistentPathExists(path: String): Unit = { - throw new UnsupportedOperationException() - } - - def createFeatureZNode(nodeContents: FeatureZNode): Unit = { - throw new UnsupportedOperationException() - } - - def updateFeatureZNode(nodeContents: FeatureZNode): Int = { - throw new UnsupportedOperationException() - } - - /** - * Deletes the given zk path recursively - * @param path - * @param expectedControllerEpochZkVersion expected controller epoch zkVersion. - * @return true if path gets deleted successfully, false if root path doesn't exist - * @throws KeeperException if there is an error while deleting the znodes - */ - def deleteRecursive(path: String, expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion): Boolean = { - throw new UnsupportedOperationException() - } - - def pathExists(path: String): Boolean = { - throw new UnsupportedOperationException() - } - - 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. - * @param partitionsToRetry 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. - */ - case class UpdateLeaderAndIsrResult( - finishedPartitions: Map[TopicPartition, Either[Exception, LeaderAndIsr]], - partitionsToRetry: Seq[TopicPartition] - ) } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 6633db51ad2a1..e1bd1df2e551b 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -16,72 +16,8 @@ */ package kafka.zk -import java.nio.charset.StandardCharsets.UTF_8 -import java.util -import java.util.Properties -import com.fasterxml.jackson.annotation.JsonProperty -import com.fasterxml.jackson.core.JsonProcessingException -import kafka.cluster.{Broker, EndPoint} -import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} -import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment} -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} -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} -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.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.config.ConfigType - -import scala.beans.BeanProperty -import scala.collection.{Map, Seq, immutable, mutable} -import scala.jdk.CollectionConverters._ -import scala.util.{Failure, Success, Try} - // This file contains objects for encoding/decoding data stored in ZooKeeper nodes (znodes). -object ControllerZNode { - def path = "/controller" - def encode(brokerId: Int, timestamp: Long, kraftControllerEpoch: Int = -1): Array[Byte] = { - Json.encodeAsBytes(Map( - "version" -> 2, - "brokerid" -> brokerId, - "timestamp" -> timestamp.toString, - "kraftControllerEpoch" -> kraftControllerEpoch).asJava) - } - def decode(bytes: Array[Byte]): Option[Int] = Json.parseBytes(bytes).map { js => - js.asJsonObject("brokerid").to[Int] - } - def decodeController(bytes: Array[Byte], zkVersion: Int): ZKControllerRegistration = Json.tryParseBytes(bytes) match { - case Right(json) => - val controller = json.asJsonObject - val brokerId = controller("brokerid").to[Int] - val kraftControllerEpoch = controller.get("kraftControllerEpoch").map(j => j.to[Int]) - ZKControllerRegistration(brokerId, kraftControllerEpoch, zkVersion) - - case Left(err) => - throw new KafkaException(s"Failed to parse ZooKeeper registration for controller: ${new String(bytes, UTF_8)}", err) - } -} - -case class ZKControllerRegistration(broker: Int, kraftEpoch: Option[Int], zkVersion: Int) - -object ControllerEpochZNode { - def path = "/controller_epoch" - def encode(epoch: Int): Array[Byte] = epoch.toString.getBytes(UTF_8) - def decode(bytes: Array[Byte]): Int = new String(bytes, UTF_8).toInt -} - object ConfigZNode { def path = "/config" } @@ -90,301 +26,12 @@ object BrokersZNode { def path = "/brokers" } -object BrokerIdsZNode { - def path = s"${BrokersZNode.path}/ids" - def encode: Array[Byte] = null -} - -object BrokerInfo { - - /** - * - Create a broker info with v5 json format if the metadataVersion is 2.7.x or above. - * - Create a broker info with v4 json format (which includes multiple endpoints and rack) if - * the metadataVersion is 0.10.0.X or above but lesser than 2.7.x. - * - Register the broker with v2 json format otherwise. - * - * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2. - * - * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X or above - * without having to upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in - * any case). - */ - def apply(broker: Broker, metadataVersion: MetadataVersion, jmxPort: Int): BrokerInfo = { - throw new UnsupportedOperationException() - } -} - -case class BrokerInfo(broker: Broker, version: Int, jmxPort: Int) { - val path: String = BrokerIdZNode.path(broker.id) - def toJsonBytes: Array[Byte] = BrokerIdZNode.encode(this) -} - -object BrokerIdZNode { - private val HostKey = "host" - private val PortKey = "port" - private val VersionKey = "version" - private val EndpointsKey = "endpoints" - private val RackKey = "rack" - private val JmxPortKey = "jmx_port" - private val ListenerSecurityProtocolMapKey = "listener_security_protocol_map" - private val TimestampKey = "timestamp" - private val FeaturesKey = "features" - - def path(id: Int) = s"${BrokerIdsZNode.path}/$id" - - /** - * Encode to JSON bytes. - * - * The JSON format includes a top level host and port for compatibility with older clients. - */ - def encode(version: Int, host: String, port: Int, advertisedEndpoints: Seq[EndPoint], jmxPort: Int, - rack: Option[String], features: Features[SupportedVersionRange]): Array[Byte] = { - val jsonMap = collection.mutable.Map(VersionKey -> version, - HostKey -> host, - PortKey -> port, - EndpointsKey -> advertisedEndpoints.map(_.connectionString).toBuffer.asJava, - JmxPortKey -> jmxPort, - TimestampKey -> Time.SYSTEM.milliseconds().toString - ) - rack.foreach(rack => if (version >= 3) jsonMap += (RackKey -> rack)) - - if (version >= 4) { - jsonMap += (ListenerSecurityProtocolMapKey -> advertisedEndpoints.map { endPoint => - endPoint.listenerName.value -> endPoint.securityProtocol.name - }.toMap.asJava) - } - - if (version >= 5) { - jsonMap += (FeaturesKey -> features.toMap) - } - Json.encodeAsBytes(jsonMap.asJava) - } - - def encode(brokerInfo: BrokerInfo): Array[Byte] = { - val broker = brokerInfo.broker - // the default host and port are here for compatibility with older clients that only support PLAINTEXT - // we choose the first plaintext port, if there is one - // or we register an empty endpoint, which means that older clients will not be able to connect - val plaintextEndpoint = broker.endPoints.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).getOrElse( - new EndPoint(null, -1, null, null)) - encode(brokerInfo.version, plaintextEndpoint.host, plaintextEndpoint.port, broker.endPoints, brokerInfo.jmxPort, - broker.rack, broker.features) - } - - private def featuresAsJavaMap(brokerInfo: JsonObject): util.Map[String, util.Map[String, java.lang.Short]] = { - FeatureZNode.asJavaMap(brokerInfo - .get(FeaturesKey) - .flatMap(_.to[Option[Map[String, Map[String, Int]]]]) - .map(theMap => theMap.map { - case(featureName, versionsInfo) => featureName -> versionsInfo.map { - case(label, version) => label -> version.asInstanceOf[Short] - }.toMap - }.toMap) - .getOrElse(Map[String, Map[String, Short]]())) - } - - /** - * Create a BrokerInfo object from id and JSON bytes. - * - * @param id - * @param jsonBytes - * - * Version 1 JSON schema for a broker is: - * { - * "version":1, - * "host":"localhost", - * "port":9092 - * "jmx_port":9999, - * "timestamp":"2233345666" - * } - * - * Version 2 JSON schema for a broker is: - * { - * "version":2, - * "host":"localhost", - * "port":9092, - * "jmx_port":9999, - * "timestamp":"2233345666", - * "endpoints":["PLAINTEXT://host1:9092", "SSL://host1:9093"] - * } - * - * Version 3 JSON schema for a broker is: - * { - * "version":3, - * "host":"localhost", - * "port":9092, - * "jmx_port":9999, - * "timestamp":"2233345666", - * "endpoints":["PLAINTEXT://host1:9092", "SSL://host1:9093"], - * "rack":"dc1" - * } - * - * Version 4 JSON schema for a broker is: - * { - * "version":4, - * "host":"localhost", - * "port":9092, - * "jmx_port":9999, - * "timestamp":"2233345666", - * "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], - * "rack":"dc1" - * } - * - * Version 5 (current) JSON schema for a broker is: - * { - * "version":5, - * "host":"localhost", - * "port":9092, - * "jmx_port":9999, - * "timestamp":"2233345666", - * "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], - * "rack":"dc1", - * "features": {"feature": {"min_version":1, "first_active_version":2, "max_version":3}} - * } - */ - def decode(id: Int, jsonBytes: Array[Byte]): BrokerInfo = { - Json.tryParseBytes(jsonBytes) match { - case Right(js) => - val brokerInfo = js.asJsonObject - val version = brokerInfo(VersionKey).to[Int] - val jmxPort = brokerInfo(JmxPortKey).to[Int] - - val endpoints = - if (version < 1) - throw new KafkaException("Unsupported version of broker registration: " + - s"${new String(jsonBytes, UTF_8)}") - else if (version == 1) { - val host = brokerInfo(HostKey).to[String] - val port = brokerInfo(PortKey).to[Int] - val securityProtocol = SecurityProtocol.PLAINTEXT - val endPoint = new EndPoint(host, port, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol) - Seq(endPoint) - } - else { - val securityProtocolMap = brokerInfo.get(ListenerSecurityProtocolMapKey) match { - case None => SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO - case Some(m) => { - val result = new java.util.HashMap[ListenerName, SecurityProtocol]() - m.to[Map[String, String]].foreach { - case (k, v) => result.put( - new ListenerName(k), SecurityProtocol.forName(v)) - } - result - } - } - val listenersString = brokerInfo(EndpointsKey).to[Seq[String]].mkString(",") - SocketServerConfigs.listenerListToEndPoints(listenersString, securityProtocolMap). - asScala.map(EndPoint.fromJava(_)) - } - - val rack = brokerInfo.get(RackKey).flatMap(_.to[Option[String]]) - val features = featuresAsJavaMap(brokerInfo) - BrokerInfo( - Broker(id, endpoints, rack, fromSupportedFeaturesMap(features)), version, jmxPort) - case Left(e) => - throw new KafkaException(s"Failed to parse ZooKeeper registration for broker $id: " + - s"${new String(jsonBytes, UTF_8)}", e) - } - } -} - object TopicsZNode { def path = s"${BrokersZNode.path}/topics" } object TopicZNode { - case class TopicIdReplicaAssignment(topic: String, - topicId: Option[Uuid], - assignment: Map[TopicPartition, ReplicaAssignment]) def path(topic: String) = s"${TopicsZNode.path}/$topic" - def encode(topicId: Option[Uuid], - assignment: collection.Map[TopicPartition, ReplicaAssignment]): Array[Byte] = { - val replicaAssignmentJson = mutable.Map[String, util.List[Int]]() - val addingReplicasAssignmentJson = mutable.Map[String, util.List[Int]]() - val removingReplicasAssignmentJson = mutable.Map[String, util.List[Int]]() - - for ((partition, replicaAssignment) <- assignment) { - replicaAssignmentJson += (partition.partition.toString -> replicaAssignment.replicas.asJava) - if (replicaAssignment.addingReplicas.nonEmpty) - addingReplicasAssignmentJson += (partition.partition.toString -> replicaAssignment.addingReplicas.asJava) - if (replicaAssignment.removingReplicas.nonEmpty) - removingReplicasAssignmentJson += (partition.partition.toString -> replicaAssignment.removingReplicas.asJava) - } - - val topicAssignment = mutable.Map( - "version" -> 3, - "partitions" -> replicaAssignmentJson.asJava, - "adding_replicas" -> addingReplicasAssignmentJson.asJava, - "removing_replicas" -> removingReplicasAssignmentJson.asJava - ) - topicId.foreach(id => topicAssignment += "topic_id" -> id.toString) - - Json.encodeAsBytes(topicAssignment.asJava) - } - def decode(topic: String, bytes: Array[Byte]): TopicIdReplicaAssignment = { - def getReplicas(replicasJsonOpt: Option[JsonObject], partition: String): Seq[Int] = { - replicasJsonOpt match { - case Some(replicasJson) => replicasJson.get(partition) match { - case Some(ar) => ar.to[Seq[Int]] - case None => Seq.empty[Int] - } - case None => Seq.empty[Int] - } - } - - Json.parseBytes(bytes).map { js => - val assignmentJson = js.asJsonObject - val topicId = assignmentJson.get("topic_id").map(_.to[String]).map(Uuid.fromString) - val addingReplicasJsonOpt = assignmentJson.get("adding_replicas").map(_.asJsonObject) - val removingReplicasJsonOpt = assignmentJson.get("removing_replicas").map(_.asJsonObject) - val partitionsJsonOpt = assignmentJson.get("partitions").map(_.asJsonObject) - val partitions = partitionsJsonOpt.map { partitionsJson => - partitionsJson.iterator.map { case (partition, replicas) => - new TopicPartition(topic, partition.toInt) -> ReplicaAssignment( - replicas.to[Seq[Int]], - getReplicas(addingReplicasJsonOpt, partition), - getReplicas(removingReplicasJsonOpt, partition) - ) - }.toMap - }.getOrElse(immutable.Map.empty[TopicPartition, ReplicaAssignment]) - - TopicIdReplicaAssignment(topic, topicId, partitions) - }.getOrElse(TopicIdReplicaAssignment(topic, None, Map.empty[TopicPartition, ReplicaAssignment])) - } -} - -object TopicPartitionsZNode { - def path(topic: String) = s"${TopicZNode.path(topic)}/partitions" -} - -object TopicPartitionZNode { - def path(partition: TopicPartition) = s"${TopicPartitionsZNode.path(partition.topic)}/${partition.partition}" -} - -object TopicPartitionStateZNode { - def path(partition: TopicPartition) = s"${TopicPartitionZNode.path(partition)}/state" - - def encode(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch): Array[Byte] = { - val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - val controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - var partitionState = Map( - "version" -> 1, - "leader" -> leaderAndIsr.leader, - "leader_epoch" -> leaderAndIsr.leaderEpoch, - "controller_epoch" -> controllerEpoch, - "isr" -> leaderAndIsr.isr - ) - - if (leaderAndIsr.leaderRecoveryState != LeaderRecoveryState.RECOVERED) { - partitionState = partitionState ++ Seq("leader_recovery_state" -> leaderAndIsr.leaderRecoveryState.value.toInt) - } - - Json.encodeAsBytes(partitionState.asJava) - } - - def decode(bytes: Array[Byte], stat: Stat): Option[LeaderIsrAndControllerEpoch] = { - throw new UnsupportedOperationException() - } } object ConfigEntityTypeZNode { @@ -393,430 +40,10 @@ object ConfigEntityTypeZNode { object ConfigEntityZNode { def path(entityType: String, entityName: String) = s"${ConfigEntityTypeZNode.path(entityType)}/$entityName" - def encode(config: Properties): Array[Byte] = { - Json.encodeAsBytes(Map("version" -> 1, "config" -> config).asJava) - } - def decode(bytes: Array[Byte]): Properties = { - val props = new Properties() - if (bytes != null) { - Json.parseBytes(bytes).foreach { js => - val configOpt = js.asJsonObjectOption.flatMap(_.get("config").flatMap(_.asJsonObjectOption)) - configOpt.foreach(config => config.iterator.foreach { case (k, v) => props.setProperty(k, v.to[String]) }) - } - } - props - } -} - -object ConfigEntityChangeNotificationZNode { - def path = s"${ConfigZNode.path}/changes" -} - -object ConfigEntityChangeNotificationSequenceZNode { - val SequenceNumberPrefix = "config_change_" - def createPath = s"${ConfigEntityChangeNotificationZNode.path}/$SequenceNumberPrefix" - def encode(sanitizedEntityPath: String): Array[Byte] = Json.encodeAsBytes( - Map("version" -> 2, "entity_path" -> sanitizedEntityPath).asJava) -} - -object IsrChangeNotificationZNode { - def path = "/isr_change_notification" -} - -object IsrChangeNotificationSequenceZNode { - val SequenceNumberPrefix = "isr_change_" - def path(sequenceNumber: String = "") = s"${IsrChangeNotificationZNode.path}/$SequenceNumberPrefix$sequenceNumber" - def encode(partitions: collection.Set[TopicPartition]): Array[Byte] = { - val partitionsJson = partitions.map(partition => Map("topic" -> partition.topic, "partition" -> partition.partition).asJava) - Json.encodeAsBytes(Map("version" -> IsrChangeNotificationHandler.Version, "partitions" -> partitionsJson.asJava).asJava) - } - - def decode(bytes: Array[Byte]): Set[TopicPartition] = { - Json.parseBytes(bytes).map { js => - val partitionsJson = js.asJsonObject("partitions").asJsonArray - partitionsJson.iterator.map { partitionsJson => - val partitionJson = partitionsJson.asJsonObject - val topic = partitionJson("topic").to[String] - val partition = partitionJson("partition").to[Int] - new TopicPartition(topic, partition) - } - } - }.map(_.toSet).getOrElse(Set.empty) - def sequenceNumber(path: String): String = path.substring(path.lastIndexOf(SequenceNumberPrefix) + SequenceNumberPrefix.length) -} - -object LogDirEventNotificationZNode { - def path = "/log_dir_event_notification" -} - -object LogDirEventNotificationSequenceZNode { - val SequenceNumberPrefix = "log_dir_event_" - private val LogDirFailureEvent = 1 - def path(sequenceNumber: String) = s"${LogDirEventNotificationZNode.path}/$SequenceNumberPrefix$sequenceNumber" - def encode(brokerId: Int): Array[Byte] = { - Json.encodeAsBytes(Map("version" -> 1, "broker" -> brokerId, "event" -> LogDirFailureEvent).asJava) - } - def decode(bytes: Array[Byte]): Option[Int] = Json.parseBytes(bytes).map { js => - js.asJsonObject("broker").to[Int] - } - def sequenceNumber(path: String): String = path.substring(path.lastIndexOf(SequenceNumberPrefix) + SequenceNumberPrefix.length) -} - -object AdminZNode { - def path = "/admin" -} - -object DeleteTopicsZNode { - def path = s"${AdminZNode.path}/delete_topics" -} - -object DeleteTopicsTopicZNode { - def path(topic: String) = s"${DeleteTopicsZNode.path}/$topic" -} - -/** - * The znode for initiating a partition reassignment. - * @deprecated Since 2.4, use the PartitionReassignment Kafka API instead. - */ -object ReassignPartitionsZNode { - - /** - * The assignment of brokers for a `TopicPartition`. - * - * A replica assignment consists of a `topic`, `partition` and a list of `replicas`, which - * represent the broker ids that the `TopicPartition` is assigned to. - */ - case class ReplicaAssignment(@BeanProperty @JsonProperty("topic") topic: String, - @BeanProperty @JsonProperty("partition") partition: Int, - @BeanProperty @JsonProperty("replicas") replicas: java.util.List[Int]) - - /** - * An assignment consists of a `version` and a list of `partitions`, which represent the - * assignment of topic-partitions to brokers. - * @deprecated Use the PartitionReassignment Kafka API instead - */ - @Deprecated - case class LegacyPartitionAssignment(@BeanProperty @JsonProperty("version") version: Int, - @BeanProperty @JsonProperty("partitions") partitions: java.util.List[ReplicaAssignment]) - - def path = s"${AdminZNode.path}/reassign_partitions" - - def encode(reassignmentMap: collection.Map[TopicPartition, Seq[Int]]): Array[Byte] = { - val reassignment = LegacyPartitionAssignment(1, - reassignmentMap.toSeq.map { case (tp, replicas) => - ReplicaAssignment(tp.topic, tp.partition, replicas.asJava) - }.asJava - ) - Json.encodeAsBytes(reassignment) - } - - def decode(bytes: Array[Byte]): Either[JsonProcessingException, collection.Map[TopicPartition, Seq[Int]]] = - Json.parseBytesAs[LegacyPartitionAssignment](bytes).map { partitionAssignment => - partitionAssignment.partitions.asScala.iterator.map { replicaAssignment => - new TopicPartition(replicaAssignment.topic, replicaAssignment.partition) -> replicaAssignment.replicas.asScala - }.toMap - } -} - -object PreferredReplicaElectionZNode { - def path = s"${AdminZNode.path}/preferred_replica_election" - def encode(partitions: Set[TopicPartition]): Array[Byte] = { - val jsonMap = Map("version" -> 1, - "partitions" -> partitions.map(tp => Map("topic" -> tp.topic, "partition" -> tp.partition).asJava).asJava) - Json.encodeAsBytes(jsonMap.asJava) - } - def decode(bytes: Array[Byte]): Set[TopicPartition] = Json.parseBytes(bytes).map { js => - val partitionsJson = js.asJsonObject("partitions").asJsonArray - partitionsJson.iterator.map { partitionsJson => - val partitionJson = partitionsJson.asJsonObject - val topic = partitionJson("topic").to[String] - val partition = partitionJson("partition").to[Int] - new TopicPartition(topic, partition) - } - }.map(_.toSet).getOrElse(Set.empty) -} - -//old consumer path znode -object ConsumerPathZNode { - def path = "/consumers" -} - -object ConsumerOffset { - def path(group: String, topic: String, partition: Integer) = s"${ConsumerPathZNode.path}/$group/offsets/$topic/$partition" - def encode(offset: Long): Array[Byte] = offset.toString.getBytes(UTF_8) - def decode(bytes: Array[Byte]): Option[Long] = Option(bytes).map(new String(_, UTF_8).toLong) } object ZkVersion { val MatchAnyVersion: Int = -1 // if used in a conditional set, matches any version (the value should match ZooKeeper codebase) - val UnknownVersion: Int = -2 // Version returned from get if node does not exist (internal constant for Kafka codebase, unused value in ZK) -} - -object ZkStat { - val NoStat = new Stat() -} - -object StateChangeHandlers { - val ControllerHandler = "controller-state-change-handler" - def zkNodeChangeListenerHandler(seqNodeRoot: String) = s"change-notification-$seqNodeRoot" -} - -/** - * Acls for resources are stored in ZK under two root paths: - *
    - *
  • [[org.apache.kafka.common.resource.PatternType#LITERAL Literal]] patterns are stored under '/kafka-acl'. - * The format is JSON. See [[kafka.zk.ResourceZNode]] for details.
  • - *
  • All other patterns are stored under '/kafka-acl-extended/pattern-type'. - * The format is JSON. See [[kafka.zk.ResourceZNode]] for details.
  • - *
- * - * Under each root node there will be one child node per resource type (Topic, Cluster, Group, etc). - * Under each resourceType there will be a unique child for each resource pattern and the data for that child will contain - * list of its acls as a json object. Following gives an example: - * - *
-  * // Literal patterns:
-  * /kafka-acl/Topic/topic-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
-  * /kafka-acl/Cluster/kafka-cluster => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
-  *
-  * // Prefixed patterns:
-  * /kafka-acl-extended/PREFIXED/Group/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]}
-  * 
- * - * Acl change events are also stored under two paths: - *
    - *
  • [[org.apache.kafka.common.resource.PatternType#LITERAL Literal]] patterns are stored under '/kafka-acl-changes'. - * The format is a UTF8 string in the form: <resource-type>:<resource-name>
  • - *
  • All other patterns are stored under '/kafka-acl-extended-changes' - * The format is JSON, as defined by [[kafka.zk.ExtendedAclChangeEvent]]
  • - *
- */ -sealed trait ZkAclStore { - val patternType: PatternType - val aclPath: String - - def path(resourceType: ResourceType): String = s"$aclPath/${SecurityUtils.resourceTypeName(resourceType)}" - - def path(resourceType: ResourceType, resourceName: String): String = s"$aclPath/${SecurityUtils.resourceTypeName(resourceType)}/$resourceName" - - def changeStore: ZkAclChangeStore -} - -object ZkAclStore { - private val storesByType: Map[PatternType, ZkAclStore] = PatternType.values - .filter(_.isSpecific) - .map(patternType => (patternType, create(patternType))) - .toMap - - val stores: Iterable[ZkAclStore] = storesByType.values - - val securePaths: Iterable[String] = stores - .flatMap(store => Set(store.aclPath, store.changeStore.aclChangePath)) - - def apply(patternType: PatternType): ZkAclStore = { - storesByType.get(patternType) match { - case Some(store) => store - case None => throw new KafkaException(s"Invalid pattern type: $patternType") - } - } - - private def create(patternType: PatternType) = { - patternType match { - case PatternType.LITERAL => LiteralAclStore - case _ => new ExtendedAclStore(patternType) - } - } -} - -object LiteralAclStore extends ZkAclStore { - val patternType: PatternType = PatternType.LITERAL - val aclPath: String = "/kafka-acl" - - def changeStore: ZkAclChangeStore = LiteralAclChangeStore -} - -class ExtendedAclStore(val patternType: PatternType) extends ZkAclStore { - if (patternType == PatternType.LITERAL) - throw new IllegalArgumentException("Literal pattern types are not supported") - - val aclPath: String = s"${ExtendedAclZNode.path}/${patternType.name.toLowerCase}" - - def changeStore: ZkAclChangeStore = ExtendedAclChangeStore -} - -object ExtendedAclZNode { - def path = "/kafka-acl-extended" -} - -trait AclChangeNotificationHandler { - def processNotification(resource: ResourcePattern): Unit -} - -trait AclChangeSubscription extends AutoCloseable { - def close(): Unit -} - -case class AclChangeNode(path: String, bytes: Array[Byte]) - -sealed trait ZkAclChangeStore { - val aclChangePath: String - def createPath: String = s"$aclChangePath/${ZkAclChangeStore.SequenceNumberPrefix}" - - def decode(bytes: Array[Byte]): ResourcePattern - - protected def encode(resource: ResourcePattern): Array[Byte] - - def createChangeNode(resource: ResourcePattern): AclChangeNode = AclChangeNode(createPath, encode(resource)) - - def createListener(handler: AclChangeNotificationHandler, zkClient: KafkaZkClient): AclChangeSubscription = { - val rawHandler: NotificationHandler = (bytes: Array[Byte]) => handler.processNotification(decode(bytes)) - - val aclChangeListener = new ZkNodeChangeNotificationListener( - zkClient, aclChangePath, ZkAclChangeStore.SequenceNumberPrefix, rawHandler) - - aclChangeListener.init() - - () => aclChangeListener.close() - } -} - -object ZkAclChangeStore { - val stores: Iterable[ZkAclChangeStore] = List(LiteralAclChangeStore, ExtendedAclChangeStore) - - def SequenceNumberPrefix = "acl_changes_" -} - -case object LiteralAclChangeStore extends ZkAclChangeStore { - val name = "LiteralAclChangeStore" - val aclChangePath: String = "/kafka-acl-changes" - - def encode(resource: ResourcePattern): Array[Byte] = { - if (resource.patternType != PatternType.LITERAL) - throw new IllegalArgumentException("Only literal resource patterns can be encoded") - - val legacyName = resource.resourceType.toString + AclEntry.RESOURCE_SEPARATOR + resource.name - legacyName.getBytes(UTF_8) - } - - def decode(bytes: Array[Byte]): ResourcePattern = { - val string = new String(bytes, UTF_8) - string.split(AclEntry.RESOURCE_SEPARATOR, 2) match { - case Array(resourceType, resourceName, _*) => new ResourcePattern(ResourceType.fromString(resourceType), resourceName, PatternType.LITERAL) - case _ => throw new IllegalArgumentException("expected a string in format ResourceType:ResourceName but got " + string) - } - } -} - -case object ExtendedAclChangeStore extends ZkAclChangeStore { - val name = "ExtendedAclChangeStore" - val aclChangePath: String = "/kafka-acl-extended-changes" - - def encode(resource: ResourcePattern): Array[Byte] = { - if (resource.patternType == PatternType.LITERAL) - throw new IllegalArgumentException("Literal pattern types are not supported") - - Json.encodeAsBytes(ExtendedAclChangeEvent( - ExtendedAclChangeEvent.currentVersion, - resource.resourceType.name, - resource.name, - resource.patternType.name)) - } - - def decode(bytes: Array[Byte]): ResourcePattern = { - val changeEvent = Json.parseBytesAs[ExtendedAclChangeEvent](bytes) match { - case Right(event) => event - case Left(e) => throw new IllegalArgumentException("Failed to parse ACL change event", e) - } - - changeEvent.toResource match { - case Success(r) => r - case Failure(e) => throw new IllegalArgumentException("Failed to convert ACL change event to resource", e) - } - } -} - -object ResourceZNode { - def path(resource: ResourcePattern): String = ZkAclStore(resource.patternType).path(resource.resourceType, resource.name) - - def encode(acls: Set[AclEntry]): Array[Byte] = { - throw new UnsupportedOperationException() - } - - def decode(bytes: Array[Byte], stat: Stat): ZkData.VersionedAcls = { - throw new UnsupportedOperationException() - } -} - -object ExtendedAclChangeEvent { - val currentVersion: Int = 1 -} - -case class ExtendedAclChangeEvent(@BeanProperty @JsonProperty("version") version: Int, - @BeanProperty @JsonProperty("resourceType") resourceType: String, - @BeanProperty @JsonProperty("name") name: String, - @BeanProperty @JsonProperty("patternType") patternType: String) { - if (version > ExtendedAclChangeEvent.currentVersion) - throw new UnsupportedVersionException(s"Acl change event received for unsupported version: $version") - - def toResource: Try[ResourcePattern] = { - for { - resType <- Try(ResourceType.fromString(resourceType)) - patType <- Try(PatternType.fromString(patternType)) - resource = new ResourcePattern(resType, name, patType) - } yield resource - } -} - -object ClusterZNode { - def path = "/cluster" -} - -object ClusterIdZNode { - def path = s"${ClusterZNode.path}/id" - - def toJson(id: String): Array[Byte] = { - Json.encodeAsBytes(Map("version" -> "1", "id" -> id).asJava) - } - - def fromJson(clusterIdJson: Array[Byte]): String = { - Json.parseBytes(clusterIdJson).map(_.asJsonObject("id").to[String]).getOrElse { - throw new KafkaException(s"Failed to parse the cluster id json ${clusterIdJson.mkString("Array(", ", ", ")")}") - } - } -} - -object BrokerSequenceIdZNode { - def path = s"${BrokersZNode.path}/seqid" -} - -object ProducerIdBlockZNode { - val CurrentVersion: Long = 1L - - def path = "/latest_producer_id_block" - - def generateProducerIdBlockJson(producerIdBlock: ProducerIdsBlock): Array[Byte] = { - Json.encodeAsBytes(Map("version" -> CurrentVersion, - "broker" -> producerIdBlock.assignedBrokerId, - "block_start" -> producerIdBlock.firstProducerId.toString, - "block_end" -> producerIdBlock.lastProducerId.toString).asJava - ) - } - - def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdsBlock = { - val jsonDataAsString = jsonData.map(_.toChar).mkString - try { - Json.parseBytes(jsonData).map(_.asJsonObject).flatMap { js => - val brokerId = js("broker").to[Int] - val blockStart = js("block_start").to[String].toLong - val blockEnd = js("block_end").to[String].toLong - Some(new ProducerIdsBlock(brokerId, blockStart, Math.toIntExact(blockEnd - blockStart + 1))) - }.getOrElse(throw new KafkaException(s"Failed to parse the producerId block json $jsonDataAsString")) - } catch { - case e: java.lang.NumberFormatException => - // this should never happen: the written data has exceeded long type limit - throw new KafkaException(s"Read jason data $jsonDataAsString contains producerIds that have exceeded long type limit", e) - } - } } object DelegationTokenAuthZNode { @@ -829,238 +56,8 @@ object DelegationTokenChangeNotificationZNode { object DelegationTokenChangeNotificationSequenceZNode { val SequenceNumberPrefix = "token_change_" - def createPath = s"${DelegationTokenChangeNotificationZNode.path}/$SequenceNumberPrefix" - def deletePath(sequenceNode: String) = s"${DelegationTokenChangeNotificationZNode.path}/$sequenceNode" - def encode(tokenId : String): Array[Byte] = tokenId.getBytes(UTF_8) - def decode(bytes: Array[Byte]): String = new String(bytes, UTF_8) } object DelegationTokensZNode { def path = s"${DelegationTokenAuthZNode.path}/tokens" } - -object DelegationTokenInfoZNode { - def path(tokenId: String) = s"${DelegationTokensZNode.path}/$tokenId" - def encode(tokenInfo: TokenInformation): Array[Byte] = - Json.encodeAsBytes(DelegationTokenManagerZk.toJsonCompatibleMap(tokenInfo).asJava) - def decode(bytes: Array[Byte]): Option[TokenInformation] = DelegationTokenManagerZk.fromBytes(bytes) -} - -/** - * Represents the status of the FeatureZNode. - * - * Enabled -> This status means the feature versioning system (KIP-584) is enabled, and, the - * finalized features stored in the FeatureZNode are active. This status is written by - * the controller to the FeatureZNode only when the broker IBP config is greater than - * or equal to IBP_2_7_IV0. - * - * Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the - * the finalized features stored in the FeatureZNode is not relevant. This status is - * written by the controller to the FeatureZNode only when the broker IBP config - * is less than IBP_2_7_IV0. - */ -sealed trait FeatureZNodeStatus { - def id: Int -} - -object FeatureZNodeStatus { - case object Disabled extends FeatureZNodeStatus { - val id: Int = 0 - } - - case object Enabled extends FeatureZNodeStatus { - val id: Int = 1 - } - - def withNameOpt(id: Int): Option[FeatureZNodeStatus] = { - id match { - case Disabled.id => Some(Disabled) - case Enabled.id => Some(Enabled) - case _ => Option.empty - } - } -} - -/** - * Represents the contents of the ZK node containing finalized feature information. - * - * @param version the version of ZK node, we removed min_version_level in version 2 - * @param status the status of the ZK node - * @param features the cluster-wide finalized features - */ -case class FeatureZNode(version: Int, status: FeatureZNodeStatus, features: Map[String, Short]) { -} - -object FeatureZNode { - private val VersionKey = "version" - private val StatusKey = "status" - private val FeaturesKey = "features" - private val V1MinVersionKey = "min_version_level" - private val V1MaxVersionKey = "max_version_level" - - // V1 contains 'version', 'status' and 'features' keys. - val V1 = 1 - // V2 removes min_version_level - val V2 = 2 - - /** - * - Create a feature info with v1 json format if if the metadataVersion is before 3.2.0 - * - Create a feature znode with v2 json format if the metadataVersion is 3.2.1 or above. - */ - def apply(metadataVersion: MetadataVersion, status: FeatureZNodeStatus, features: Map[String, Short]): FeatureZNode = { - val version = if (metadataVersion.isAtLeast(MetadataVersion.IBP_3_3_IV0)) { - V2 - } else { - V1 - } - FeatureZNode(version, status, features) - } - - def path = "/feature" - - def asJavaMap(scalaMap: Map[String, Map[String, Short]]): util.Map[String, util.Map[String, java.lang.Short]] = { - scalaMap - .map { - case(featureName, versionInfo) => featureName -> versionInfo.map { - case(label, version) => label -> java.lang.Short.valueOf(version) - }.asJava - }.asJava - } - - /** - * Encodes a FeatureZNode to JSON. - * - * @param featureZNode FeatureZNode to be encoded - * - * @return JSON representation of the FeatureZNode, as an Array[Byte] - */ - def encode(featureZNode: FeatureZNode): Array[Byte] = { - val features = if (featureZNode.version == V1) { - asJavaMap(featureZNode.features.map{ - case (feature, version) => feature -> Map(V1MaxVersionKey -> version, V1MinVersionKey -> version) - }) - } else { - asJavaMap(featureZNode.features.map{ - case (feature, version) => feature -> Map(V1MaxVersionKey -> version) - }) - } - val jsonMap = collection.mutable.Map( - VersionKey -> featureZNode.version, - StatusKey -> featureZNode.status.id, - FeaturesKey -> features) - Json.encodeAsBytes(jsonMap.asJava) - } - - /** - * Decodes the contents of the feature ZK node from Array[Byte] to a FeatureZNode. - * - * @param jsonBytes the contents of the feature ZK node - * - * @return the FeatureZNode created from jsonBytes - * - * @throws IllegalArgumentException if the Array[Byte] can not be decoded. - */ - def decode(jsonBytes: Array[Byte]): FeatureZNode = { - Json.tryParseBytes(jsonBytes) match { - case Right(js) => - val featureInfo = js.asJsonObject - val version = featureInfo(VersionKey).to[Int] - if (version < V1 || version > V2) { - throw new IllegalArgumentException(s"Unsupported version: $version of feature information: " + - s"${new String(jsonBytes, UTF_8)}") - } - - val statusInt = featureInfo - .get(StatusKey) - .flatMap(_.to[Option[Int]]) - if (statusInt.isEmpty) { - throw new IllegalArgumentException("Status can not be absent in feature information: " + - s"${new String(jsonBytes, UTF_8)}") - } - val status = FeatureZNodeStatus.withNameOpt(statusInt.get) - if (status.isEmpty) { - throw new IllegalArgumentException( - s"Malformed status: $statusInt found in feature information: ${new String(jsonBytes, UTF_8)}") - } - - val finalizedFeatures = decodeFeature(version, featureInfo, jsonBytes) - FeatureZNode(version, status.get, finalizedFeatures) - case Left(e) => - throw new IllegalArgumentException(s"Failed to parse feature information: " + - s"${new String(jsonBytes, UTF_8)}", e) - } - } - - private def decodeFeature(version: Int, featureInfo: JsonObject, jsonBytes: Array[Byte]): Map[String, Short] = { - val featuresMap = featureInfo - .get(FeaturesKey) - .flatMap(_.to[Option[Map[String, Map[String, Int]]]]) - - if (featuresMap.isEmpty) { - throw new IllegalArgumentException("Features map can not be absent in: " + - s"${new String(jsonBytes, UTF_8)}") - } - featuresMap.get.map { - case (featureName, versionInfo) => - if (version == V1 && !versionInfo.contains(V1MinVersionKey)) { - throw new IllegalArgumentException(s"$V1MinVersionKey absent in [$versionInfo]") - } - if (!versionInfo.contains(V1MaxVersionKey)) { - throw new IllegalArgumentException(s"$V1MaxVersionKey absent in [$versionInfo]") - } - - val minValueOpt = versionInfo.get(V1MinVersionKey) - val maxValue = versionInfo(V1MaxVersionKey) - - if (version == V1 && (minValueOpt.get < 1 || maxValue < minValueOpt.get)) { - throw new IllegalArgumentException(s"Expected minValue >= 1, maxValue >= 1 and maxValue >= minValue, but received minValue: ${minValueOpt.get}, maxValue: $maxValue") - } - if (maxValue < 1) { - throw new IllegalArgumentException(s"Expected maxValue >= 1, but received maxValue: $maxValue") - } - featureName -> maxValue.toShort - } - } -} - - -object ZkData { - - case class VersionedAcls(acls: Set[AclEntry], zkVersion: Int) { - def exists: Boolean = zkVersion != ZkVersion.UnknownVersion - } - - - // Important: it is necessary to add any new top level Zookeeper path to the Seq - val SecureRootPaths: Seq[String] = Seq(AdminZNode.path, - BrokersZNode.path, - ClusterZNode.path, - ConfigZNode.path, - ControllerZNode.path, - ControllerEpochZNode.path, - IsrChangeNotificationZNode.path, - ProducerIdBlockZNode.path, - LogDirEventNotificationZNode.path, - DelegationTokenAuthZNode.path, - ExtendedAclZNode.path, - FeatureZNode.path) ++ ZkAclStore.securePaths - - // These are persistent ZK paths that should exist on kafka broker startup. - val PersistentZkPaths: Seq[String] = Seq( - ConsumerPathZNode.path, // old consumer path - BrokerIdsZNode.path, - TopicsZNode.path, - ConfigEntityChangeNotificationZNode.path, - DeleteTopicsZNode.path, - BrokerSequenceIdZNode.path, - IsrChangeNotificationZNode.path, - ProducerIdBlockZNode.path, - LogDirEventNotificationZNode.path - ) ++ ConfigType.ALL.asScala.map(ConfigEntityTypeZNode.path) - - val SensitiveRootPaths: Seq[String] = Seq( - ConfigEntityTypeZNode.path(ConfigType.USER), - ConfigEntityTypeZNode.path(ConfigType.BROKER), - DelegationTokensZNode.path - ) -} diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index a8a66dbd54f5c..72b76cd152d25 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -17,8 +17,6 @@ package kafka.server -import kafka.controller.ControllerEventManager - import java.io.File import java.net.InetSocketAddress import java.util @@ -393,7 +391,7 @@ object QuorumTestHarness { // when broker ports are reused (e.g. auto-create topics) as well as threads // which reset static JAAS configuration. val unexpectedThreadNames = Set( - ControllerEventManager.ControllerEventThreadName, + "controller-event-thread", KafkaProducer.NETWORK_THREAD_PREFIX, AdminClientUnitTestEnv.kafkaAdminClientNetworkThreadPrefix(), AbstractCoordinator.HEARTBEAT_THREAD_PREFIX, diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala deleted file mode 100644 index ca9179049532a..0000000000000 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ /dev/null @@ -1,204 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.cluster - -import java.nio.charset.StandardCharsets -import kafka.zk.BrokerIdZNode -import org.apache.kafka.common.feature.{Features, SupportedVersionRange} -import org.apache.kafka.common.feature.Features._ -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.server.network.BrokerEndPoint -import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals} -import org.junit.jupiter.api.Test - -import scala.jdk.CollectionConverters._ - -class BrokerEndPointTest { - - @Test - def testHashAndEquals(): Unit = { - val broker1 = new BrokerEndPoint(1, "myhost", 9092) - val broker2 = new BrokerEndPoint(1, "myhost", 9092) - val broker3 = new BrokerEndPoint(2, "myhost", 1111) - val broker4 = new BrokerEndPoint(1, "other", 1111) - - assertEquals(broker1, broker2) - assertNotEquals(broker1, broker3) - assertNotEquals(broker1, broker4) - assertEquals(broker1.hashCode, broker2.hashCode) - assertNotEquals(broker1.hashCode, broker3.hashCode) - assertNotEquals(broker1.hashCode, broker4.hashCode) - - assertEquals(Some(1), Map(broker1 -> 1).get(broker1)) - } - - @Test - def testFromJsonFutureVersion(): Unit = { - // Future compatible versions should be supported, we use a hypothetical future version here - val brokerInfoStr = """{ - "foo":"bar", - "version":100, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"1416974968782", - "endpoints":["SSL://localhost:9093"] - }""" - val broker = parseBrokerJson(1, brokerInfoStr) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.SSL)) - assertEquals("localhost", brokerEndPoint.host) - assertEquals(9093, brokerEndPoint.port) - } - - @Test - def testFromJsonV2(): Unit = { - val brokerInfoStr = """{ - "version":2, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"1416974968782", - "endpoints":["PLAINTEXT://localhost:9092"] - }""" - val broker = parseBrokerJson(1, brokerInfoStr) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) - assertEquals("localhost", brokerEndPoint.host) - assertEquals(9092, brokerEndPoint.port) - } - - @Test - def testFromJsonV1(): Unit = { - val brokerInfoStr = """{"jmx_port":-1,"timestamp":"1420485325400","host":"172.16.8.243","version":1,"port":9091}""" - val broker = parseBrokerJson(1, brokerInfoStr) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) - assertEquals("172.16.8.243", brokerEndPoint.host) - assertEquals(9091, brokerEndPoint.port) - } - - @Test - def testFromJsonV3(): Unit = { - val json = """{ - "version":3, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"2233345666", - "endpoints":["PLAINTEXT://host1:9092", "SSL://host1:9093"], - "rack":"dc1" - }""" - val broker = parseBrokerJson(1, json) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.SSL)) - assertEquals("host1", brokerEndPoint.host) - assertEquals(9093, brokerEndPoint.port) - assertEquals(Some("dc1"), broker.rack) - } - - @Test - def testFromJsonV4WithNullRack(): Unit = { - val json = """{ - "version":4, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"2233345666", - "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], - "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"}, - "rack":null - }""" - val broker = parseBrokerJson(1, json) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(new ListenerName("CLIENT")) - assertEquals("host1", brokerEndPoint.host) - assertEquals(9092, brokerEndPoint.port) - assertEquals(None, broker.rack) - } - - @Test - def testFromJsonV4WithNoRack(): Unit = { - val json = """{ - "version":4, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"2233345666", - "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], - "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"} - }""" - val broker = parseBrokerJson(1, json) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(new ListenerName("CLIENT")) - assertEquals("host1", brokerEndPoint.host) - assertEquals(9092, brokerEndPoint.port) - assertEquals(None, broker.rack) - } - - @Test - def testFromJsonV4WithNoFeatures(): Unit = { - val json = """{ - "version":4, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"2233345666", - "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], - "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"}, - "rack":"dc1" - }""" - val broker = parseBrokerJson(1, json) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(new ListenerName("CLIENT")) - assertEquals("host1", brokerEndPoint.host) - assertEquals(9092, brokerEndPoint.port) - assertEquals(Some("dc1"), broker.rack) - assertEquals(emptySupportedFeatures, broker.features) - } - - @Test - def testFromJsonV5(): Unit = { - val json = """{ - "version":5, - "host":"localhost", - "port":9092, - "jmx_port":9999, - "timestamp":"2233345666", - "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], - "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"}, - "rack":"dc1", - "features": {"feature1": {"min_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "max_version": 4}} - }""" - val broker = parseBrokerJson(1, json) - assertEquals(1, broker.id) - val brokerEndPoint = broker.brokerEndPoint(new ListenerName("CLIENT")) - assertEquals("host1", brokerEndPoint.host) - assertEquals(9092, brokerEndPoint.port) - assertEquals(Some("dc1"), broker.rack) - assertEquals(Features.supportedFeatures( - Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 2), - "feature2" -> new SupportedVersionRange(2, 4)).asJava), - broker.features) - } - - private def parseBrokerJson(id: Int, jsonString: String): Broker = - BrokerIdZNode.decode(id, jsonString.getBytes(StandardCharsets.UTF_8)).broker -} diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index b7dd63daa4518..98edbec7cbbc6 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -800,7 +800,7 @@ class LogManagerTest { val newProperties = new Properties() newProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) - spyLogManager.updateTopicConfig(topic, newProperties, isRemoteLogStorageSystemEnabled = false, wasRemoteLogEnabled = false, fromZK = false) + spyLogManager.updateTopicConfig(topic, newProperties, isRemoteLogStorageSystemEnabled = false, wasRemoteLogEnabled = false) assertTrue(log0.config.delete) assertTrue(log1.config.delete) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index fcb2ffcbb7d54..27f746a6b6d61 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -497,7 +497,7 @@ class DynamicConfigChangeUnitTest { @Test def shouldParseReplicationQuotaProperties(): Unit = { - val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null, null) + val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null) val props: Properties = new Properties() //Given @@ -510,7 +510,7 @@ class DynamicConfigChangeUnitTest { @Test def shouldParseWildcardReplicationQuotaProperties(): Unit = { - val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null, null) + val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null) val props: Properties = new Properties() //Given @@ -530,7 +530,7 @@ class DynamicConfigChangeUnitTest { CoreUtils.propsWith(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, value), 102, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG) } - val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null, null) + val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null) assertEquals(ReplicationQuotaManager.ALL_REPLICAS.asScala.map(_.toInt).toSeq, parse(configHandler, "* ")) assertEquals(Seq(), parse(configHandler, " ")) assertEquals(Seq(6), parse(configHandler, "6:102")) @@ -540,7 +540,7 @@ class DynamicConfigChangeUnitTest { @Test def shouldParseReplicationQuotaReset(): Unit = { - val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null, null) + val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null) val props: Properties = new Properties() //Given @@ -587,7 +587,7 @@ class DynamicConfigChangeUnitTest { doNothing().when(rlm).onLeadershipChange(leaderPartitionsArg.capture(), followerPartitionsArg.capture(), any()) val isRemoteLogEnabledBeforeUpdate = false - val configHandler: TopicConfigHandler = new TopicConfigHandler(replicaManager, null, null, None) + val configHandler: TopicConfigHandler = new TopicConfigHandler(replicaManager, null, null) configHandler.maybeUpdateRemoteLogComponents(topic, Seq(log0, log1), isRemoteLogEnabledBeforeUpdate, false) assertEquals(Collections.singleton(partition0), leaderPartitionsArg.getValue) assertEquals(Collections.singleton(partition1), followerPartitionsArg.getValue) @@ -611,7 +611,7 @@ class DynamicConfigChangeUnitTest { when(partition.isLeader).thenReturn(true) val isRemoteLogEnabledBeforeUpdate = true - val configHandler: TopicConfigHandler = new TopicConfigHandler(replicaManager, null, null, None) + val configHandler: TopicConfigHandler = new TopicConfigHandler(replicaManager, null, null) configHandler.maybeUpdateRemoteLogComponents(topic, Seq(log0), isRemoteLogEnabledBeforeUpdate, false) verify(rlm, never()).onLeadershipChange(any(), any(), any()) }