diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index a167756..36ddeb4 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -140,8 +140,7 @@ object AdminUtils extends Logging { } def deleteTopic(zkClient: ZkClient, topic: String) { - zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) - zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) + ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) } def topicExists(zkClient: ZkClient, topic: String): Boolean = diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 842c110..65510eb 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -118,7 +118,7 @@ object TopicCommand { val topics = getTopics(zkClient, opts) topics.foreach { topic => AdminUtils.deleteTopic(zkClient, topic) - println("Topic \"%s\" deleted.".format(topic)) + println("Topic \"%s\" queued for deletion.".format(topic)) } } @@ -257,7 +257,6 @@ object TopicCommand { val topicsWithOverridesOpt = parser.accepts("topics-with-overrides", "if set when listing topics, only show topics that have overridden configs") - val options = parser.parse(args : _*) } diff --git a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala index a80aa49..46ec3db 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala @@ -18,11 +18,9 @@ package kafka.api import java.nio.ByteBuffer -import collection.mutable.HashMap -import collection.immutable.Map import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.api.ApiUtils._ - +import collection.Set object ControlledShutdownResponse { def readFrom(buffer: ByteBuffer): ControlledShutdownResponse = { diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index a984878..0311737 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -26,6 +26,7 @@ import kafka.controller.LeaderIsrAndControllerEpoch import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping import kafka.network.RequestChannel.Response +import collection.Set object LeaderAndIsr { diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index 820f0f5..68fc138 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -17,13 +17,13 @@ package kafka.api - import java.nio._ import kafka.api.ApiUtils._ import kafka.network.{BoundedByteBufferSend, RequestChannel, InvalidRequestException} -import kafka.common.ErrorMapping +import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.network.RequestChannel.Response import kafka.utils.Logging +import collection.Set object StopReplicaRequest extends Logging { @@ -44,9 +44,9 @@ object StopReplicaRequest extends Logging { throw new InvalidRequestException("Invalid byte %d in delete partitions field. (Assuming false.)".format(x)) } val topicPartitionPairCount = buffer.getInt - val topicPartitionPairSet = new collection.mutable.HashSet[(String, Int)]() + val topicPartitionPairSet = new collection.mutable.HashSet[TopicAndPartition]() (1 to topicPartitionPairCount) foreach { _ => - topicPartitionPairSet.add(readShortString(buffer), buffer.getInt) + topicPartitionPairSet.add(TopicAndPartition(readShortString(buffer), buffer.getInt)) } StopReplicaRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, deletePartitions, topicPartitionPairSet.toSet) @@ -59,10 +59,10 @@ case class StopReplicaRequest(versionId: Short, controllerId: Int, controllerEpoch: Int, deletePartitions: Boolean, - partitions: Set[(String, Int)]) + partitions: Set[TopicAndPartition]) extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) { - def this(deletePartitions: Boolean, partitions: Set[(String, Int)], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { + def this(deletePartitions: Boolean, partitions: Set[TopicAndPartition], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, controllerId, controllerEpoch, deletePartitions, partitions) } @@ -75,9 +75,9 @@ case class StopReplicaRequest(versionId: Short, buffer.putInt(controllerEpoch) buffer.put(if (deletePartitions) 1.toByte else 0.toByte) buffer.putInt(partitions.size) - for ((topic, partitionId) <- partitions){ - writeShortString(buffer, topic) - buffer.putInt(partitionId) + for (topicAndPartition <- partitions) { + writeShortString(buffer, topicAndPartition.topic) + buffer.putInt(topicAndPartition.partition) } } @@ -90,8 +90,8 @@ case class StopReplicaRequest(versionId: Short, 4 + /* controller epoch */ 1 + /* deletePartitions */ 4 /* partition count */ - for ((topic, partitionId) <- partitions){ - size += (ApiUtils.shortStringLength(topic)) + + for (topicAndPartition <- partitions){ + size += (ApiUtils.shortStringLength(topicAndPartition.topic)) + 4 /* partition id */ } size diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala index d7e3630..c90ddee 100644 --- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala +++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala @@ -20,7 +20,7 @@ package kafka.api import java.nio.ByteBuffer import collection.mutable.HashMap import collection.immutable.Map -import kafka.common.ErrorMapping +import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.api.ApiUtils._ @@ -30,12 +30,12 @@ object StopReplicaResponse { val errorCode = buffer.getShort val numEntries = buffer.getInt - val responseMap = new HashMap[(String, Int), Short]() + val responseMap = new HashMap[TopicAndPartition, Short]() for (i<- 0 until numEntries){ val topic = readShortString(buffer) val partition = buffer.getInt val partitionErrorCode = buffer.getShort() - responseMap.put((topic, partition), partitionErrorCode) + responseMap.put(TopicAndPartition(topic, partition), partitionErrorCode) } new StopReplicaResponse(correlationId, responseMap.toMap, errorCode) } @@ -43,7 +43,7 @@ object StopReplicaResponse { case class StopReplicaResponse(override val correlationId: Int, - val responseMap: Map[(String, Int), Short], + val responseMap: Map[TopicAndPartition, Short], val errorCode: Short = ErrorMapping.NoError) extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int ={ @@ -53,7 +53,7 @@ case class StopReplicaResponse(override val correlationId: Int, 4 /* number of responses */ for ((key, value) <- responseMap) { size += - 2 + key._1.length /* topic */ + + 2 + key.topic.length /* topic */ + 4 /* partition */ + 2 /* error code for this partition */ } @@ -64,10 +64,10 @@ case class StopReplicaResponse(override val correlationId: Int, buffer.putInt(correlationId) buffer.putShort(errorCode) buffer.putInt(responseMap.size) - for ((key:(String, Int), value) <- responseMap){ - writeShortString(buffer, key._1) - buffer.putInt(key._2) - buffer.putShort(value) + for ((topicAndPartition, errorCode) <- responseMap){ + writeShortString(buffer, topicAndPartition.topic) + buffer.putInt(topicAndPartition.partition) + buffer.putShort(errorCode) } } diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 54dd7bd..543e262 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -22,6 +22,7 @@ import kafka.cluster.Broker import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response +import collection.Set object UpdateMetadataRequest { val CurrentVersion = 0.shortValue diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index ea8485b..a1ee5a7 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -25,7 +25,10 @@ import kafka.server.KafkaConfig import collection.mutable import kafka.api._ import org.apache.log4j.Logger +import scala.Some import kafka.common.TopicAndPartition +import kafka.api.RequestOrResponse +import collection.Set class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging { private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] @@ -118,10 +121,8 @@ class RequestSendThread(val controllerId: Int, val queueItem = queue.take() val request = queueItem._1 val callback = queueItem._2 - var receive: Receive = null - - try{ + try { lock synchronized { var isSendSuccessful = false while(isRunning.get() && !isSendSuccessful) { @@ -155,7 +156,7 @@ class RequestSendThread(val controllerId: Int, stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %s" .format(controllerId, controllerContext.epoch, response.correlationId, toBroker.toString())) - if(callback != null){ + if(callback != null) { callback(response) } } @@ -180,12 +181,12 @@ class RequestSendThread(val controllerId: Int, } } -class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendRequest: (Int, RequestOrResponse, (RequestOrResponse) => Unit) => Unit, - controllerId: Int, clientId: String) - extends Logging { +class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging { + val controllerContext = controller.controllerContext + val controllerId: Int = controller.config.brokerId + val clientId: String = controller.clientId val leaderAndIsrRequestMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), PartitionStateInfo]] - val stopReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]] - val stopAndDeleteReplicaRequestMap = new mutable.HashMap[Int, Seq[(String, Int)]] + val stopReplicaRequestMap = new mutable.HashMap[Int, Seq[StopReplicaRequestInfo]] val updateMetadataRequestMap = new mutable.HashMap[Int, mutable.HashMap[TopicAndPartition, PartitionStateInfo]] private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) @@ -200,52 +201,47 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq if(updateMetadataRequestMap.size > 0) throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + "new one. Some UpdateMetadata state changes %s might be lost ".format(updateMetadataRequestMap.toString())) - if(stopAndDeleteReplicaRequestMap.size > 0) - throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + - "new one. Some StopReplica with delete state changes %s might be lost ".format(stopAndDeleteReplicaRequestMap.toString())) } def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - replicas: Seq[Int]) { - brokerIds.foreach { brokerId => + replicas: Seq[Int], callback: (RequestOrResponse) => Unit = null) { + brokerIds.filter(b => b >= 0).foreach { brokerId => leaderAndIsrRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[(String, Int), PartitionStateInfo]) leaderAndIsrRequestMap(brokerId).put((topic, partition), PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet)) } - addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(TopicAndPartition(topic, partition))) + addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq) } - def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean) { - brokerIds.foreach { brokerId => - stopReplicaRequestMap.getOrElseUpdate(brokerId, Seq.empty[(String, Int)]) - stopAndDeleteReplicaRequestMap.getOrElseUpdate(brokerId, Seq.empty[(String, Int)]) - if (deletePartition) { - val v = stopAndDeleteReplicaRequestMap(brokerId) - stopAndDeleteReplicaRequestMap(brokerId) = v :+ (topic, partition) - } - else { - val v = stopReplicaRequestMap(brokerId) - stopReplicaRequestMap(brokerId) = v :+ (topic, partition) - } + def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean, + callback: (RequestOrResponse, Int) => Unit = null) { + brokerIds.filter(b => b >= 0).foreach { brokerId => + stopReplicaRequestMap.getOrElseUpdate(brokerId, Seq.empty[StopReplicaRequestInfo]) + val v = stopReplicaRequestMap(brokerId) + if(callback != null) + stopReplicaRequestMap(brokerId) = v :+ StopReplicaRequestInfo(PartitionAndReplica(topic, partition, brokerId), + deletePartition, (r: RequestOrResponse) => { callback(r, brokerId) }) + else + stopReplicaRequestMap(brokerId) = v :+ StopReplicaRequestInfo(PartitionAndReplica(topic, partition, brokerId), + deletePartition) } } + /* Send UpdateMetadataRequest to the given brokers for all partitions except those being deleted as part of delete topic + * + */ def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], - partitions:scala.collection.Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { - val partitionList = - if(partitions.isEmpty) { - controllerContext.partitionLeadershipInfo.keySet - } else { - partitions - } + callback: (RequestOrResponse) => Unit = null) { + val partitionList = controllerContext.partitionLeadershipInfo.keySet.dropWhile( + p => controller.deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) partitionList.foreach { partition => val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) leaderIsrAndControllerEpochOpt match { case Some(leaderIsrAndControllerEpoch) => val replicas = controllerContext.partitionReplicaAssignment(partition).toSet val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) - brokerIds.foreach { brokerId => + brokerIds.filter(b => b >= 0).foreach { brokerId => updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo) } @@ -269,7 +265,7 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq p._2.leaderIsrAndControllerEpoch, correlationId, broker, p._1._1, p._1._2)) } - sendRequest(broker, leaderAndIsrRequest, null) + controller.sendRequest(broker, leaderAndIsrRequest, null) } leaderAndIsrRequestMap.clear() updateMetadataRequestMap.foreach { m => @@ -280,24 +276,23 @@ class ControllerBrokerRequestBatch(controllerContext: ControllerContext, sendReq partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, correlationId, broker, p._1))) - sendRequest(broker, updateMetadataRequest, null) + controller.sendRequest(broker, updateMetadataRequest, null) } updateMetadataRequestMap.clear() - Seq((stopReplicaRequestMap, false), (stopAndDeleteReplicaRequestMap, true)) foreach { - case(m, deletePartitions) => { - m foreach { - case(broker, replicas) => - if (replicas.size > 0) { - debug("The stop replica request (delete = %s) sent to broker %d is %s" - .format(deletePartitions, broker, replicas.mkString(","))) - val stopReplicaRequest = new StopReplicaRequest(deletePartitions, Set.empty[(String, Int)] ++ replicas, controllerId, - controllerEpoch, correlationId) - sendRequest(broker, stopReplicaRequest, null) - } - } - m.clear() + stopReplicaRequestMap foreach { case(broker, replicaInfoList) => + val stopReplicaWithDelete = replicaInfoList.filter(p => p.deletePartition == true).map(i => i.replica).toSet + val stopReplicaWithoutDelete = replicaInfoList.filter(p => p.deletePartition == false).map(i => i.replica).toSet + debug("The stop replica request (delete = true) sent to broker %d is %s" + .format(broker, stopReplicaWithDelete.mkString(","))) + debug("The stop replica request (delete = false) sent to broker %d is %s" + .format(broker, stopReplicaWithoutDelete.mkString(","))) + replicaInfoList.foreach { r => + val stopReplicaRequest = new StopReplicaRequest(r.deletePartition, + Set(TopicAndPartition(r.replica.topic, r.replica.partition)), controllerId, controllerEpoch, correlationId) + controller.sendRequest(broker, stopReplicaRequest, r.callback) } } + stopReplicaRequestMap.clear() } } @@ -306,3 +301,35 @@ case class ControllerBrokerStateInfo(channel: BlockingChannel, messageQueue: BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)], requestSendThread: RequestSendThread) +case class StopReplicaRequestInfo(replica: PartitionAndReplica, deletePartition: Boolean, callback: (RequestOrResponse) => Unit = null) + +class Callbacks private (var leaderAndIsrResponseCallback:(RequestOrResponse) => Unit = null, + var updateMetadataResponseCallback:(RequestOrResponse) => Unit = null, + var stopReplicaResponseCallback:(RequestOrResponse, Int) => Unit = null) + +object Callbacks { + class CallbackBuilder { + var leaderAndIsrResponseCbk:(RequestOrResponse) => Unit = null + var updateMetadataResponseCbk:(RequestOrResponse) => Unit = null + var stopReplicaResponseCbk:(RequestOrResponse, Int) => Unit = null + + def leaderAndIsrCallback(cbk: (RequestOrResponse) => Unit): CallbackBuilder = { + leaderAndIsrResponseCbk = cbk + this + } + + def updateMetadataCallback(cbk: (RequestOrResponse) => Unit): CallbackBuilder = { + updateMetadataResponseCbk = cbk + this + } + + def stopReplicaCallback(cbk: (RequestOrResponse, Int) => Unit): CallbackBuilder = { + stopReplicaResponseCbk = cbk + this + } + + def build: Callbacks = { + new Callbacks(leaderAndIsrResponseCbk, updateMetadataResponseCbk, stopReplicaResponseCbk) + } + } +} diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index a0267ae..2395b96 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -17,42 +17,42 @@ package kafka.controller import collection._ -import collection.immutable.Set +import collection.Set import com.yammer.metrics.core.Gauge import java.lang.{IllegalStateException, Object} import java.util.concurrent.TimeUnit -import kafka.admin.{AdminOperationException, PreferredReplicaLeaderElectionCommand} +import kafka.admin.PreferredReplicaLeaderElectionCommand import kafka.api._ import kafka.cluster.Broker import kafka.common._ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import kafka.server.{ZookeeperLeaderElector, KafkaConfig} import kafka.utils.ZkUtils._ -import kafka.utils.{Json, Utils, ZkUtils, Logging, KafkaScheduler} +import kafka.utils._ +import kafka.utils.Utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} import java.util.concurrent.atomic.AtomicInteger +import org.apache.log4j.Logger import scala.Some import kafka.common.TopicAndPartition -import org.apache.log4j.Logger +import java.util.concurrent.locks.ReentrantLock class ControllerContext(val zkClient: ZkClient, - val zkSessionTimeout: Int, - var controllerChannelManager: ControllerChannelManager = null, - val controllerLock: Object = new Object, - var shuttingDownBrokerIds: mutable.Set[Int] = mutable.Set.empty, - val brokerShutdownLock: Object = new Object, - var epoch: Int = KafkaController.InitialControllerEpoch - 1, - var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion - 1, - val correlationId: AtomicInteger = new AtomicInteger(0), - var allTopics: Set[String] = Set.empty, - var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty, - var partitionLeadershipInfo: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty, - var partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = - new mutable.HashMap, - var partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = - new mutable.HashSet) { + val zkSessionTimeout: Int) { + var controllerChannelManager: ControllerChannelManager = null + val controllerLock: ReentrantLock = new ReentrantLock() + var shuttingDownBrokerIds: mutable.Set[Int] = mutable.Set.empty + val brokerShutdownLock: Object = new Object + var epoch: Int = KafkaController.InitialControllerEpoch - 1 + var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion - 1 + val correlationId: AtomicInteger = new AtomicInteger(0) + var allTopics: Set[String] = Set.empty + var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty + var partitionLeadershipInfo: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty + var partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap + var partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = new mutable.HashSet private var liveBrokersUnderlying: Set[Broker] = Set.empty private var liveBrokerIdsUnderlying: Set[Int] = Set.empty @@ -86,9 +86,37 @@ class ControllerContext(val zkClient: ZkClient, }.flatten.toSet } + def replicasForTopic(topic: String): Set[PartitionAndReplica] = { + partitionReplicaAssignment + .filter { case(topicAndPartition, replicas) => topicAndPartition.topic.equals(topic) } + .map { case(topicAndPartition, replicas) => + replicas.map { r => + new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, r) + } + }.flatten.toSet + } + + def partitionsForTopic(topic: String): collection.Set[TopicAndPartition] = { + partitionReplicaAssignment + .filter { case(topicAndPartition, replicas) => topicAndPartition.topic.equals(topic) }.keySet + } + def allLiveReplicas(): Set[PartitionAndReplica] = { replicasOnBrokers(liveBrokerIds) } + + def replicasForPartition(partitions: collection.Set[TopicAndPartition]): collection.Set[PartitionAndReplica] = { + partitions.map { p => + val replicas = partitionReplicaAssignment(p) + replicas.map(r => new PartitionAndReplica(p.topic, p.partition, r)) + }.flatten + } + + def removeTopic(topic: String) = { + partitionLeadershipInfo = partitionLeadershipInfo.dropWhile(p => p._1.topic.equals(topic)) + partitionReplicaAssignment = partitionReplicaAssignment.dropWhile(p => p._1.topic.equals(topic)) + allTopics -= topic + } } trait KafkaControllerMBean { @@ -128,18 +156,19 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private var isRunning = true private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) val controllerContext = new ControllerContext(zkClient, config.zkSessionTimeoutMs) - private val partitionStateMachine = new PartitionStateMachine(this) - private val replicaStateMachine = new ReplicaStateMachine(this) + val partitionStateMachine = new PartitionStateMachine(this) + val replicaStateMachine = new ReplicaStateMachine(this) private val controllerElector = new ZookeeperLeaderElector(controllerContext, ZkUtils.ControllerPath, onControllerFailover, onControllerResignation, config.brokerId) // have a separate scheduler for the controller to be able to start and stop independently of the // kafka server private val autoRebalanceScheduler = new KafkaScheduler(1) + var deleteTopicManager: TopicDeletionManager = null val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - private val brokerRequestBatch = new ControllerBrokerRequestBatch(controllerContext, sendRequest, this.config.brokerId, this.clientId) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(this) registerControllerChangedListener() newGauge( @@ -153,7 +182,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg "OfflinePartitionsCount", new Gauge[Int] { def value(): Int = { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { if (!isActive()) 0 else @@ -167,7 +196,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg "PreferredReplicaImbalanceCount", new Gauge[Int] { def value(): Int = { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { if (!isActive()) 0 else @@ -200,7 +229,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.brokerShutdownLock synchronized { info("Shutting down broker " + id) - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { if (!controllerContext.liveOrShuttingDownBrokerIds.contains(id)) throw new BrokerNotAvailableException("Broker id %d does not exist.".format(id)) @@ -211,7 +240,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } val allPartitionsAndReplicationFactorOnBroker: Set[(TopicAndPartition, Int)] = - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { controllerContext.partitionsOnBroker(id) .map(topicAndPartition => (topicAndPartition, controllerContext.partitionReplicaAssignment(topicAndPartition).size)) } @@ -219,7 +248,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg allPartitionsAndReplicationFactorOnBroker.foreach { case(topicAndPartition, replicationFactor) => // Move leadership serially to relinquish lock. - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { // If the broker leads the topic partition, transition the leader and update isr. Updates zk and @@ -231,7 +260,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // Stop the replica first. The state change below initiates ZK changes which should take some time // before which the stop replica request should be completed (in most cases) brokerRequestBatch.newBatch() - brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, topicAndPartition.partition, deletePartition = false) + brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, + topicAndPartition.partition, deletePartition = false) brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) // If the broker is a follower, updates the isr in ZK and notifies the current leader @@ -242,7 +272,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def replicatedPartitionsBrokerLeads() = controllerContext.controllerLock.synchronized { + def replicatedPartitionsBrokerLeads() = inLock(controllerContext.controllerLock) { trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(",")) controllerContext.partitionLeadershipInfo.filter { case (topicAndPartition, leaderIsrAndControllerEpoch) => @@ -283,8 +313,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.allTopics.foreach(topic => partitionStateMachine.registerPartitionChangeListener(topic)) Utils.registerMBean(this, KafkaController.MBeanName) info("Broker %d is ready to serve as the new controller with epoch %d".format(config.brokerId, epoch)) - initializeAndMaybeTriggerPartitionReassignment() - initializeAndMaybeTriggerPreferredReplicaElection() + maybeTriggerPartitionReassignment() + maybeTriggerPreferredReplicaElection() /* send partition leadership info to all live brokers */ sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) if (config.autoLeaderRebalanceEnable) { @@ -293,6 +323,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg autoRebalanceScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) } + deleteTopicManager.start() } else info("Controller has been shut down, aborting startup/failover") @@ -303,7 +334,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * required to clean up internal controller data structures */ def onControllerResignation() { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { + autoRebalanceScheduler.shutdown() + deleteTopicManager.shutdown() Utils.unregisterMBean(KafkaController.MBeanName) partitionStateMachine.shutdown() replicaStateMachine.shutdown() @@ -318,7 +351,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * Returns true if this broker is the current controller. */ def isActive(): Boolean = { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { controllerContext.controllerChannelManager != null } } @@ -338,7 +371,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg */ def onBrokerStartup(newBrokers: Seq[Int]) { info("New broker startup callback for %s".format(newBrokers.mkString(","))) - val newBrokersSet = newBrokers.toSet // send update metadata request for all partitions to the newly restarted brokers. 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 @@ -346,16 +378,25 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg sendUpdateMetadataRequest(newBrokers) // 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 - replicaStateMachine.handleStateChanges(controllerContext.replicasOnBrokers(newBrokersSet), OnlineReplica) + val allReplicasOnNewBrokers = controllerContext.replicasOnBrokers(newBrokersSet) + replicaStateMachine.handleStateChanges(allReplicasOnNewBrokers, 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 - val partitionsWithReplicasOnNewBrokers = controllerContext.partitionsBeingReassigned.filter{ - case (topicAndPartition, reassignmentContext) => - reassignmentContext.newReplicas.exists(newBrokersSet.contains(_)) + val partitionsWithReplicasOnNewBrokers = controllerContext.partitionsBeingReassigned.filter { + case (topicAndPartition, reassignmentContext) => reassignmentContext.newReplicas.exists(newBrokersSet.contains(_)) } partitionsWithReplicasOnNewBrokers.foreach(p => onPartitionReassignment(p._1, p._2)) + // 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 => deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) + if(replicasForTopicsToBeDeleted.size > 0) { + info(("Some replicas %s for topics scheduled for deletion %s are on the newly restarted brokers %s. " + + "Signaling restart of topic deletion for these topics").format(replicasForTopicsToBeDeleted.mkString(","), + deleteTopicManager.topicsToBeDeleted.mkString(","), newBrokers.mkString(","))) + deleteTopicManager.resumeDeletionForTopics(replicasForTopicsToBeDeleted.map(_.topic)) + } } /** @@ -371,20 +412,30 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg */ def onBrokerFailure(deadBrokers: Seq[Int]) { info("Broker failure callback for %s".format(deadBrokers.mkString(","))) - val deadBrokersThatWereShuttingDown = deadBrokers.filter(id => controllerContext.shuttingDownBrokerIds.remove(id)) info("Removed %s from list of shutting down brokers.".format(deadBrokersThatWereShuttingDown)) - val deadBrokersSet = deadBrokers.toSet // trigger OfflinePartition state for all partitions whose current leader is one amongst the dead brokers val partitionsWithoutLeader = controllerContext.partitionLeadershipInfo.filter(partitionAndLeader => - deadBrokersSet.contains(partitionAndLeader._2.leaderAndIsr.leader)).keySet + deadBrokersSet.contains(partitionAndLeader._2.leaderAndIsr.leader) && + !deleteTopicManager.isTopicQueuedUpForDeletion(partitionAndLeader._1.topic)).keySet partitionStateMachine.handleStateChanges(partitionsWithoutLeader, OfflinePartition) // trigger OnlinePartition state changes for offline or new partitions partitionStateMachine.triggerOnlinePartitionStateChange() + // filter out the replicas that belong to topics that are being deleted + var allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokersSet) + val activeReplicasOnDeadBrokers = allReplicasOnDeadBrokers.filterNot(p => deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) // handle dead replicas - replicaStateMachine.handleStateChanges(controllerContext.replicasOnBrokers(deadBrokersSet), OfflineReplica) + replicaStateMachine.handleStateChanges(activeReplicasOnDeadBrokers, OfflineReplica) + // check if topic deletion state for the dead replicas needs to be updated + val replicasForTopicsToBeDeleted = allReplicasOnDeadBrokers.filter(p => deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) + if(replicasForTopicsToBeDeleted.size > 0) { + // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be + // deleted when the broker is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely + // since topic deletion cannot be retried if at least one replica is in TopicDeletionStarted state + deleteTopicManager.failReplicaDeletion(replicasForTopicsToBeDeleted) + } } /** @@ -401,7 +452,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } /** - * This callback is invoked by the partition state machine's partition change listener with the list of new partitions. + * 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 @@ -409,9 +460,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg def onNewPartitionCreation(newPartitions: Set[TopicAndPartition]) { info("New partition creation callback for %s".format(newPartitions.mkString(","))) partitionStateMachine.handleStateChanges(newPartitions, NewPartition) - replicaStateMachine.handleStateChanges(getAllReplicasForPartition(newPartitions), NewReplica) + replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions), NewReplica) partitionStateMachine.handleStateChanges(newPartitions, OnlinePartition, offlinePartitionSelector) - replicaStateMachine.handleStateChanges(getAllReplicasForPartition(newPartitions), OnlineReplica) + replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions), OnlineReplica) } /** @@ -493,8 +544,10 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg removePartitionFromReassignedPartitions(topicAndPartition) info("Removed partition %s from the list of reassigned partitions in zookeeper".format(topicAndPartition)) controllerContext.partitionsBeingReassigned.remove(topicAndPartition) - //12. After electing leader, the replicas and isr information changes. So resend the update metadata request to every broker. - sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicAndPartition)) + //12. After electing leader, the replicas and isr information changes, so resend the update metadata request to every broker + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + // signal delete topic thread if reassignment for some partitions belonging to topics being deleted just completed + deleteTopicManager.resumeDeletionForTopics(Set(topicAndPartition.topic)) } } @@ -528,6 +581,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // first register ISR change listener watchIsrChangesForReassignedPartition(topic, partition, reassignedPartitionContext) controllerContext.partitionsBeingReassigned.put(topicAndPartition, reassignedPartitionContext) + // halt topic deletion for the partitions being reassigned + deleteTopicManager.haltTopicDeletion(Set(topic)) onPartitionReassignment(topicAndPartition, reassignedPartitionContext) } else { // some replica in RAR is not alive. Fail partition reassignment @@ -550,11 +605,15 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions + info(("Halting deletion of topics %s at this time since preferred replica election is in progress for " + + "partitions %s").format(partitions.map(_.topic).mkString(","), partitions.mkString(","))) + deleteTopicManager.haltTopicDeletion(partitions.map(_.topic)) partitionStateMachine.handleStateChanges(partitions, OnlinePartition, preferredReplicaPartitionLeaderSelector) } catch { case e: Throwable => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) } finally { removePartitionsFromPreferredReplicaElection(partitions) + deleteTopicManager.resumeDeletionForTopics(partitions.map(_.topic)) } } @@ -564,7 +623,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * elector */ def startup() = { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { info("Controller starting up"); registerSessionExpirationListener() isRunning = true @@ -579,7 +638,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * shuts down the controller channel manager, if one exists (i.e. if it was the current controller) */ def shutdown() = { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { isRunning = false partitionStateMachine.shutdown() replicaStateMachine.shutdown() @@ -633,6 +692,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } private def initializeControllerContext() { + // update controller cache with delete topic information controllerContext.liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).toSet controllerContext.allTopics = ZkUtils.getAllTopics(zkClient).toSet controllerContext.partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, controllerContext.allTopics.toSeq) @@ -642,42 +702,74 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg updateLeaderAndIsrCache() // start the channel manager startChannelManager() + initializePreferredReplicaElection() + initializePartitionReassignment() + initializeTopicDeletion() info("Currently active brokers in the cluster: %s".format(controllerContext.liveBrokerIds)) info("Currently shutting brokers in the cluster: %s".format(controllerContext.shuttingDownBrokerIds)) info("Current list of topics in the cluster: %s".format(controllerContext.allTopics)) } - private def initializeAndMaybeTriggerPartitionReassignment() { + private def initializePreferredReplicaElection() { + // initialize preferred replica election state + val partitionsUndergoingPreferredReplicaElection = ZkUtils.getPartitionsUndergoingPreferredReplicaElection(zkClient) + // check if they are already completed or topic was deleted + val partitionsThatCompletedPreferredReplicaElection = partitionsUndergoingPreferredReplicaElection.filter { partition => + val replicasOpt = controllerContext.partitionReplicaAssignment.get(partition) + val topicDeleted = replicasOpt.isEmpty + val successful = + if(!topicDeleted) controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader == replicasOpt.get.head else false + successful || topicDeleted + } + controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitionsUndergoingPreferredReplicaElection + controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsThatCompletedPreferredReplicaElection + info("Partitions undergoing preferred replica election: %s".format(partitionsUndergoingPreferredReplicaElection.mkString(","))) + info("Partitions that completed preferred replica election: %s".format(partitionsThatCompletedPreferredReplicaElection.mkString(","))) + info("Resuming preferred replica election for partitions: %s".format(controllerContext.partitionsUndergoingPreferredReplicaElection.mkString(","))) + } + + private def initializePartitionReassignment() { // read the partitions being reassigned from zookeeper path /admin/reassign_partitions val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient) - // check if they are already completed - val reassignedPartitions = partitionsBeingReassigned.filter(partition => - controllerContext.partitionReplicaAssignment(partition._1) == partition._2.newReplicas).map(_._1) + // check if they are already completed or topic was deleted + val reassignedPartitions = partitionsBeingReassigned.filter { partition => + val replicasOpt = controllerContext.partitionReplicaAssignment.get(partition._1) + val topicDeleted = replicasOpt.isEmpty + val successful = if(!topicDeleted) replicasOpt.get == partition._2.newReplicas else false + topicDeleted || successful + }.map(_._1) reassignedPartitions.foreach(p => removePartitionFromReassignedPartitions(p)) var partitionsToReassign: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap partitionsToReassign ++= partitionsBeingReassigned partitionsToReassign --= reassignedPartitions - + controllerContext.partitionsBeingReassigned ++= partitionsToReassign info("Partitions being reassigned: %s".format(partitionsBeingReassigned.toString())) info("Partitions already reassigned: %s".format(reassignedPartitions.toString())) info("Resuming reassignment of partitions: %s".format(partitionsToReassign.toString())) + } - partitionsToReassign.foreach { topicPartitionToReassign => + private def initializeTopicDeletion() { + val topicsQueuedForDeletion = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.DeleteTopicsPath).toSet + val replicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter(r => + r._2.foldLeft(false)((res,r) => res || !controllerContext.liveBrokerIds.contains(r))) + val topicsWithReplicasOnDeadBrokers = replicasOnDeadBrokers.map(_._1.topic).toSet + val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic) + val topicsForWhichPreferredReplicaElectionIsInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic) + val haltedTopicsForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress | + topicsForWhichPreferredReplicaElectionIsInProgress + info("List of topics to be deleted: %s".format(topicsQueuedForDeletion.mkString(","))) + info("List of topics halted for deletion: %s".format(haltedTopicsForDeletion.mkString(","))) + // initialize the topic deletion manager + deleteTopicManager = new TopicDeletionManager(this, topicsQueuedForDeletion, haltedTopicsForDeletion) + } + + private def maybeTriggerPartitionReassignment() { + controllerContext.partitionsBeingReassigned.foreach { topicPartitionToReassign => initiateReassignReplicasForTopicPartition(topicPartitionToReassign._1, topicPartitionToReassign._2) } } - private def initializeAndMaybeTriggerPreferredReplicaElection() { - // read the partitions undergoing preferred replica election from zookeeper path - val partitionsUndergoingPreferredReplicaElection = ZkUtils.getPartitionsUndergoingPreferredReplicaElection(zkClient) - // check if they are already completed - val partitionsThatCompletedPreferredReplicaElection = partitionsUndergoingPreferredReplicaElection.filter(partition => - controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader == controllerContext.partitionReplicaAssignment(partition).head) - controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitionsUndergoingPreferredReplicaElection - controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsThatCompletedPreferredReplicaElection - info("Partitions undergoing preferred replica election: %s".format(partitionsUndergoingPreferredReplicaElection.mkString(","))) - info("Partitions that completed preferred replica election: %s".format(partitionsThatCompletedPreferredReplicaElection.mkString(","))) - info("Resuming preferred replica election for partitions: %s".format(controllerContext.partitionsUndergoingPreferredReplicaElection.mkString(","))) + private def maybeTriggerPreferredReplicaElection() { onPreferredReplicaElection(controllerContext.partitionsUndergoingPreferredReplicaElection.toSet) } @@ -736,13 +828,13 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg val topic = topicAndPartition.topic val partition = topicAndPartition.partition // first move the replica to offline state (the controller removes it from the ISR) - oldReplicas.foreach { replica => - replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topic, partition, replica)), OfflineReplica) - } + val replicasToBeDeleted = oldReplicas.map(r => PartitionAndReplica(topic, partition, r)) + replicaStateMachine.handleStateChanges(replicasToBeDeleted, OfflineReplica) // send stop replica command to the old replicas - oldReplicas.foreach { replica => - replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topic, partition, replica)), NonExistentReplica) - } + 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 updateAssignedReplicasForPartition(topicAndPartition: TopicAndPartition, @@ -838,22 +930,14 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsToBeRemoved } - private def getAllReplicasForPartition(partitions: Set[TopicAndPartition]): Set[PartitionAndReplica] = { - partitions.map { p => - val replicas = controllerContext.partitionReplicaAssignment(p) - replicas.map(r => new PartitionAndReplica(p.topic, p.partition, r)) - }.flatten - } - /** * 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 - * @param partitions The partitions for which the metadata is to be sent */ - private def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { + def sendUpdateMetadataRequest(brokers: Seq[Int]) { brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) + brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers) brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) } @@ -979,7 +1063,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg @throws(classOf[Exception]) def handleNewSession() { info("ZK expired; shut down all controller components and try to re-elect") - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { onControllerResignation() controllerElector.elect } @@ -991,10 +1075,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg trace("checking need to trigger partition rebalance") // get all the active brokers var preferredReplicasForTopicsByBrokers: Map[Int, Map[TopicAndPartition, Seq[Int]]] = null - controllerContext.controllerLock synchronized { - preferredReplicasForTopicsByBrokers = controllerContext.partitionReplicaAssignment.groupBy { - case(topicAndPartition, assignedReplicas) => assignedReplicas.head - } + inLock(controllerContext.controllerLock) { + preferredReplicasForTopicsByBrokers = + controllerContext.partitionReplicaAssignment.filterNot(p => deleteTopicManager.isTopicQueuedUpForDeletion(p._1.topic)).groupBy { + case(topicAndPartition, assignedReplicas) => assignedReplicas.head + } } debug("preferred replicas by broker " + preferredReplicasForTopicsByBrokers) // for each broker, check if a preferred replica election needs to be triggered @@ -1002,7 +1087,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg case(leaderBroker, topicAndPartitionsForBroker) => { var imbalanceRatio: Double = 0 var topicsNotInPreferredReplica: Map[TopicAndPartition, Seq[Int]] = null - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { topicsNotInPreferredReplica = topicAndPartitionsForBroker.filter { case(topicPartition, replicas) => { @@ -1018,7 +1103,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // 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)) { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { // do this check only if the broker is live and there are no partitions being reassigned currently // and preferred replica election is not in progress if (controllerContext.liveBrokerIds.contains(leaderBroker) && @@ -1070,11 +1155,19 @@ class PartitionsReassignedListener(controller: KafkaController) extends IZkDataL debug("Partitions reassigned listener fired for path %s. Record partitions to be reassigned %s" .format(dataPath, data)) val partitionsReassignmentData = ZkUtils.parsePartitionReassignmentData(data.toString) - val newPartitions = partitionsReassignmentData.filterNot(p => controllerContext.partitionsBeingReassigned.contains(p._1)) - newPartitions.foreach { partitionToBeReassigned => - controllerContext.controllerLock synchronized { - val context = new ReassignedPartitionsContext(partitionToBeReassigned._2) - controller.initiateReassignReplicasForTopicPartition(partitionToBeReassigned._1, context) + val partitionsToBeReassigned = inLock(controllerContext.controllerLock) { + partitionsReassignmentData.filterNot(p => controllerContext.partitionsBeingReassigned.contains(p._1)) + } + partitionsToBeReassigned.foreach { partitionToBeReassigned => + inLock(controllerContext.controllerLock) { + if(controller.deleteTopicManager.isTopicQueuedUpForDeletion(partitionToBeReassigned._1.topic)) { + error("Skipping reassignment of partition %s for topic %s since it is currently being deleted" + .format(partitionToBeReassigned._1, partitionToBeReassigned._1.topic)) + controller.removePartitionFromReassignedPartitions(partitionToBeReassigned._1) + } else { + val context = new ReassignedPartitionsContext(partitionToBeReassigned._2) + controller.initiateReassignReplicasForTopicPartition(partitionToBeReassigned._1, context) + } } } } @@ -1102,11 +1195,11 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: */ @throws(classOf[Exception]) def handleDataChange(dataPath: String, data: Object) { - try { - controllerContext.controllerLock synchronized { - debug("Reassigned partitions isr change listener fired for path %s with children %s".format(dataPath, data)) + inLock(controllerContext.controllerLock) { + debug("Reassigned partitions isr change listener fired for path %s with children %s".format(dataPath, data)) + val topicAndPartition = TopicAndPartition(topic, partition) + try { // check if this partition is still being reassigned or not - val topicAndPartition = TopicAndPartition(topic, partition) controllerContext.partitionsBeingReassigned.get(topicAndPartition) match { case Some(reassignedPartitionContext) => // need to re-read leader and isr from zookeeper since the zkclient callback doesn't return the Stat object @@ -1131,9 +1224,9 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic: } case None => } + } catch { + case e: Throwable => error("Error while handling partition reassignment", e) } - }catch { - case e: Throwable => error("Error while handling partition reassignment", e) } } @@ -1163,13 +1256,19 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD def handleDataChange(dataPath: String, data: Object) { debug("Preferred replica election listener fired for path %s. Record partitions to undergo preferred replica election %s" .format(dataPath, data.toString)) - val partitionsForPreferredReplicaElection = PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(data.toString) - - controllerContext.controllerLock synchronized { - info("These partitions are already undergoing preferred replica election: %s" - .format(controllerContext.partitionsUndergoingPreferredReplicaElection.mkString(","))) - val newPartitions = partitionsForPreferredReplicaElection -- controllerContext.partitionsUndergoingPreferredReplicaElection - controller.onPreferredReplicaElection(newPartitions) + inLock(controllerContext.controllerLock) { + val partitionsForPreferredReplicaElection = PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(data.toString) + if(controllerContext.partitionsUndergoingPreferredReplicaElection.size > 0) + info("These partitions are already undergoing preferred replica election: %s" + .format(controllerContext.partitionsUndergoingPreferredReplicaElection.mkString(","))) + var partitions = partitionsForPreferredReplicaElection -- controllerContext.partitionsUndergoingPreferredReplicaElection + val partitionsForTopicsToBeDeleted = partitions.filter(p => controller.deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) + if(partitionsForTopicsToBeDeleted.size > 0) { + error("Skipping preferred replica election for partitions %s since the respective topics are being deleted" + .format(partitionsForTopicsToBeDeleted)) + } + else + controller.onPreferredReplicaElection(partitions -- partitionsForTopicsToBeDeleted) } } @@ -1194,7 +1293,7 @@ class ControllerEpochListener(controller: KafkaController) extends IZkDataListen @throws(classOf[Exception]) def handleDataChange(dataPath: String, data: Object) { debug("Controller epoch listener fired with new epoch " + data.toString) - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { // read the epoch path to get the zk version readControllerEpochFromZookeeper() } @@ -1222,7 +1321,11 @@ class ControllerEpochListener(controller: KafkaController) extends IZkDataListen case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty, var isrChangeListener: ReassignedPartitionsIsrChangeListener = null) -case class PartitionAndReplica(topic: String, partition: Int, replica: Int) +case class PartitionAndReplica(topic: String, partition: Int, replica: Int) { + override def toString(): String = { + "[Topic=%s,Partition=%d,Replica=%d]".format(topic, partition, replica) + } +} case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controllerEpoch: Int) { override def toString(): String = { diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index fd9200f..fa29bbe 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -48,24 +48,24 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { controllerContext.partitionReplicaAssignment.get(topicAndPartition) match { case Some(assignedReplicas) => - val liveAssignedReplicasToThisPartition = assignedReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) + val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) val liveBrokersInIsr = currentLeaderAndIsr.isr.filter(r => controllerContext.liveBrokerIds.contains(r)) val currentLeaderEpoch = currentLeaderAndIsr.leaderEpoch val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion val newLeaderAndIsr = liveBrokersInIsr.isEmpty match { case true => debug("No broker in ISR is alive for %s. Pick the leader from the alive assigned replicas: %s" - .format(topicAndPartition, liveAssignedReplicasToThisPartition.mkString(","))) - liveAssignedReplicasToThisPartition.isEmpty match { + .format(topicAndPartition, liveAssignedReplicas.mkString(","))) + liveAssignedReplicas.isEmpty match { case true => throw new NoReplicaOnlineException(("No replica for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + " Assigned replicas are: [%s]".format(assignedReplicas)) case false => ControllerStats.uncleanLeaderElectionRate.mark() - val newLeader = liveAssignedReplicasToThisPartition.head + val newLeader = liveAssignedReplicas.head warn("No broker in ISR is alive for %s. Elect leader %d from live brokers %s. There's potential data loss." - .format(topicAndPartition, newLeader, liveAssignedReplicasToThisPartition.mkString(","))) + .format(topicAndPartition, newLeader, liveAssignedReplicas.mkString(","))) new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1) } case false => @@ -75,7 +75,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1) } info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition)) - (newLeaderAndIsr, liveAssignedReplicasToThisPartition) + (newLeaderAndIsr, liveAssignedReplicas) case None => throw new NoReplicaOnlineException("Partition %s doesn't have".format(topicAndPartition) + "replicas assigned to it") } @@ -106,10 +106,10 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex case None => reassignedInSyncReplicas.size match { case 0 => - throw new StateChangeFailedException("List of reassigned replicas for partition " + + throw new NoReplicaOnlineException("List of reassigned replicas for partition " + " %s is empty. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) case _ => - throw new StateChangeFailedException("None of the reassigned replicas for partition " + + throw new NoReplicaOnlineException("None of the reassigned replicas for partition " + "%s are in-sync with the leader. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) } } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index ac4262a..487d4c8 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -26,6 +26,8 @@ import kafka.utils.{Logging, ZkUtils} import org.I0Itec.zkclient.{IZkDataListener, IZkChildListener} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.apache.log4j.Logger +import kafka.controller.Callbacks.CallbackBuilder +import kafka.utils.Utils._ /** * This class represents the state machine for partitions. It defines the states that a partition can be in, and @@ -44,8 +46,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient var partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.controllerContext, controller.sendRequest, - controllerId, controller.clientId) + val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) private val hasStarted = new AtomicBoolean(false) private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext) this.logIdent = "[Partition state machine on Controller " + controllerId + "]: " @@ -68,6 +69,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { // register topic and partition change listeners def registerListeners() { registerTopicChangeListener() + registerDeleteTopicListener() } /** @@ -85,10 +87,13 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { def triggerOnlinePartitionStateChange() { try { brokerRequestBatch.newBatch() - // try to move all partitions in NewPartition or OfflinePartition state to OnlinePartition state - for((topicAndPartition, partitionState) <- partitionState) { + // try to move all partitions in NewPartition or OfflinePartition state to OnlinePartition state except partitions + // that belong to topics to be deleted + for((topicAndPartition, partitionState) <- partitionState + if(!controller.deleteTopicManager.isTopicQueuedUpForDeletion(topicAndPartition.topic))) { if(partitionState.equals(OfflinePartition) || partitionState.equals(NewPartition)) - handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, controller.offlinePartitionSelector) + handleStateChange(topicAndPartition.topic, topicAndPartition.partition, OnlinePartition, controller.offlinePartitionSelector, + (new CallbackBuilder).build) } brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) } catch { @@ -97,18 +102,23 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } } + def partitionsInState(state: PartitionState): Set[TopicAndPartition] = { + partitionState.filter(p => p._2 == state).keySet + } + /** * This API is invoked by the partition change zookeeper listener * @param partitions The list of partitions that need to be transitioned to the target state * @param targetState The state that the partitions should be moved to */ def handleStateChanges(partitions: Set[TopicAndPartition], targetState: PartitionState, - leaderSelector: PartitionLeaderSelector = noOpPartitionLeaderSelector) { + leaderSelector: PartitionLeaderSelector = noOpPartitionLeaderSelector, + callbacks: Callbacks = (new CallbackBuilder).build) { info("Invoking state change to %s for partitions %s".format(targetState, partitions.mkString(","))) try { brokerRequestBatch.newBatch() partitions.foreach { topicAndPartition => - handleStateChange(topicAndPartition.topic, topicAndPartition.partition, targetState, leaderSelector) + handleStateChange(topicAndPartition.topic, topicAndPartition.partition, targetState, leaderSelector, callbacks) } brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) }catch { @@ -131,7 +141,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * --select new leader and isr for this partition and a set of replicas to receive the LeaderAndIsr request, and write leader and isr to ZK * --for this partition, send LeaderAndIsr request to every receiving replica and UpdateMetadata request to every live broker * - * NewPartition,OnlinePartition -> OfflinePartition + * NewPartition,OnlinePartition,OfflinePartition -> OfflinePartition * --nothing other than marking partition state as Offline * * OfflinePartition -> NonExistentPartition @@ -141,7 +151,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { * @param targetState The end state that the partition should be moved to */ private def handleStateChange(topic: String, partition: Int, targetState: PartitionState, - leaderSelector: PartitionLeaderSelector) { + leaderSelector: PartitionLeaderSelector, + callbacks: Callbacks) { val topicAndPartition = TopicAndPartition(topic, partition) if (!hasStarted.get) throw new StateChangeFailedException(("Controller %d epoch %d initiated state change for partition %s to %s failed because " + @@ -178,7 +189,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { // post: partition has a leader case OfflinePartition => // pre: partition should be in New or Online state - assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition), OfflinePartition) + assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition, OfflinePartition), OfflinePartition) // should be called when the leader for a partition is no longer alive stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Online to Offline" .format(controllerId, controller.epoch, topicAndPartition)) @@ -354,6 +365,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), new AddPartitionsListener(topic)) } + private def registerDeleteTopicListener() = { + zkClient.subscribeChildChanges(ZkUtils.DeleteTopicsPath, new DeleteTopicsListener()) + } + private def getLeaderIsrAndEpochOrThrowException(topic: String, partition: Int): LeaderIsrAndControllerEpoch = { val topicAndPartition = TopicAndPartition(topic, partition) ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match { @@ -373,7 +388,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { @throws(classOf[Exception]) def handleChildChange(parentPath : String, children : java.util.List[String]) { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { if (hasStarted.get) { try { val currentChildren = { @@ -383,7 +398,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } val newTopics = currentChildren -- controllerContext.allTopics val deletedTopics = controllerContext.allTopics -- currentChildren - // val deletedPartitionReplicaAssignment = replicaAssignment.filter(p => deletedTopics.contains(p._1._1)) controllerContext.allTopics = currentChildren val addedPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, newTopics.toSeq) @@ -397,12 +411,62 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } catch { case e: Throwable => error("Error while handling new topic", e ) } - // TODO: kafka-330 Handle deleted topics } } } } + /** + * Delete topics includes the following operations - + * 1. Add the topic to be deleted to the delete topics cache, only if the topic exists + * 2. If there are topics to be deleted, it signals the delete topic thread + */ + class DeleteTopicsListener() extends IZkChildListener with Logging { + this.logIdent = "[DeleteTopicsListener on " + controller.config.brokerId + "]: " + val zkClient = controllerContext.zkClient + + /** + * Invoked when a topic is being deleted + * @throws Exception On any error. + */ + @throws(classOf[Exception]) + def handleChildChange(parentPath : String, children : java.util.List[String]) { + inLock(controllerContext.controllerLock) { + var topicsToBeDeleted = { + import JavaConversions._ + (children: Buffer[String]).toSet + } + debug("Delete topics listener fired for topics %s to be deleted".format(topicsToBeDeleted.mkString(","))) + val nonExistentTopics = topicsToBeDeleted.filter(t => !controllerContext.allTopics.contains(t)) + if(nonExistentTopics.size > 0) + warn("Ignoring request to delete non-existing topics " + nonExistentTopics.mkString(",")) + topicsToBeDeleted --= nonExistentTopics + if(topicsToBeDeleted.size > 0) { + info("Starting topic deletion for topics " + topicsToBeDeleted.mkString(",")) + // add topic to deletion list + controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) + // halt if other state changes are in progress + topicsToBeDeleted.foreach { topic => + val preferredReplicaElectionInProgress = + controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic).contains(topic) + val partitionReassignmentInProgress = + controllerContext.partitionsBeingReassigned.keySet.map(_.topic).contains(topic) + if(preferredReplicaElectionInProgress | partitionReassignmentInProgress) + controller.deleteTopicManager.haltTopicDeletion(Set(topic)) + } + } + } + } + + /** + * + * @throws Exception + * On any error. + */ + @throws(classOf[Exception]) + def handleDataDeleted(dataPath: String) { + } + } class AddPartitionsListener(topic: String) extends IZkDataListener with Logging { @@ -410,15 +474,21 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { @throws(classOf[Exception]) def handleDataChange(dataPath : String, data: Object) { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { try { info("Add Partition triggered " + data.toString + " for path " + dataPath) val partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) - val partitionsRemainingToBeAdded = partitionReplicaAssignment.filter(p => + val partitionsToBeAdded = partitionReplicaAssignment.filter(p => !controllerContext.partitionReplicaAssignment.contains(p._1)) - info("New partitions to be added [%s]".format(partitionsRemainingToBeAdded)) - if (partitionsRemainingToBeAdded.size > 0) - controller.onNewPartitionCreation(partitionsRemainingToBeAdded.keySet.toSet) + if(controller.deleteTopicManager.isTopicQueuedUpForDeletion(topic)) + error("Skipping adding partitions %s for topic %s since it is currently being deleted" + .format(partitionsToBeAdded.map(_._1.partition).mkString(","), topic)) + else { + if (partitionsToBeAdded.size > 0) { + info("New partitions to be added %s".format(partitionsToBeAdded)) + controller.onNewPartitionCreation(partitionsToBeAdded.keySet.toSet) + } + } } catch { case e: Throwable => error("Error while handling add partitions for data path " + dataPath, e ) } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 483559a..4c84f73 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -23,6 +23,8 @@ import kafka.common.{TopicAndPartition, StateChangeFailedException} import kafka.utils.{ZkUtils, Logging} import org.I0Itec.zkclient.IZkChildListener import org.apache.log4j.Logger +import kafka.controller.Callbacks._ +import kafka.utils.Utils._ /** * This class represents the state machine for replicas. It defines the states that a replica can be in, and @@ -35,15 +37,19 @@ import org.apache.log4j.Logger * Valid previous state are NewReplica, OnlineReplica or OfflineReplica * 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 - * 4. NonExistentReplica: If a replica is deleted, it is moved to this state. Valid previous state is OfflineReplica + * 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. ReplicaDeletionFailed: If replica deletion fails, it is moved to this state. Valid previous state is ReplicaDeletionStarted + * 7. NonExistentReplica: If a replica is deleted successfully, it is moved to this state. Valid previous state is + * ReplicaDeletionSuccessful */ class ReplicaStateMachine(controller: KafkaController) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkClient = controllerContext.zkClient - var replicaState: mutable.Map[(String, Int, Int), ReplicaState] = mutable.Map.empty - val brokerRequestBatch = new ControllerBrokerRequestBatch(controller.controllerContext, controller.sendRequest, - controllerId, controller.clientId) + var replicaState: mutable.Map[PartitionAndReplica, ReplicaState] = mutable.Map.empty + val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) private val hasStarted = new AtomicBoolean(false) this.logIdent = "[Replica state machine on controller " + controller.config.brokerId + "]: " private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) @@ -81,14 +87,17 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * @param targetState The state that the replicas should be moved to * The controller's allLeaders cache should have been updated before this */ - def handleStateChanges(replicas: Set[PartitionAndReplica], targetState: ReplicaState) { - info("Invoking state change to %s for replicas %s".format(targetState, replicas.mkString(","))) - try { - brokerRequestBatch.newBatch() - replicas.foreach(r => handleStateChange(r.topic, r.partition, r.replica, targetState)) - brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) - }catch { - case e: Throwable => error("Error while moving some replicas to %s state".format(targetState), e) + def handleStateChanges(replicas: Set[PartitionAndReplica], targetState: ReplicaState, + callbacks: Callbacks = (new CallbackBuilder).build) { + if(replicas.size > 0) { + info("Invoking state change to %s for replicas %s".format(targetState, replicas.mkString(","))) + try { + brokerRequestBatch.newBatch() + replicas.foreach(r => handleStateChange(r, targetState, callbacks)) + brokerRequestBatch.sendRequestsToBrokers(controller.epoch, controllerContext.correlationId.getAndIncrement) + }catch { + case e: Throwable => error("Error while moving some replicas to %s state".format(targetState), e) + } } } @@ -104,30 +113,42 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * OnlineReplica,OfflineReplica -> OnlineReplica * --send LeaderAndIsr request with current leader and isr to the new replica and UpdateMetadata request for the partition to every live broker * - * NewReplica,OnlineReplica -> OfflineReplica + * NewReplica,OnlineReplica,OfflineReplica,ReplicaDeletionFailed -> OfflineReplica * --send StopReplicaRequest to the replica (w/o deletion) * --remove this replica from the isr and send LeaderAndIsr request (with new isr) to the leader replica and UpdateMetadata request for the partition to every live broker. * - * OfflineReplica -> NonExistentReplica + * OfflineReplica -> ReplicaDeletionStarted * --send StopReplicaRequest to the replica (with deletion) * - * @param topic The topic of the replica for which the state transition is invoked - * @param partition The partition of the replica for which the state transition is invoked - * @param replicaId The replica for which the state transition is invoked + * ReplicaDeletionStarted -> ReplicaDeletionSuccessful + * -- mark the state of the replica in the state machine + * + * ReplicaDeletionStarted -> ReplicaDeletionFailed + * -- mark the state of the replica in the state machine + * + * ReplicaDeletionSuccessful -> NonExistentReplica + * -- remove the replica from the in memory partition replica assignment cache + + + * @param partitionAndReplica The replica for which the state transition is invoked * @param targetState The end state that the replica should be moved to */ - def handleStateChange(topic: String, partition: Int, replicaId: Int, targetState: ReplicaState) { + def handleStateChange(partitionAndReplica: PartitionAndReplica, targetState: ReplicaState, + callbacks: Callbacks) { + val topic = partitionAndReplica.topic + val partition = partitionAndReplica.partition + val replicaId = partitionAndReplica.replica val topicAndPartition = TopicAndPartition(topic, partition) if (!hasStarted.get) throw new StateChangeFailedException(("Controller %d epoch %d initiated state change of replica %d for partition %s " + "to %s failed because replica state machine has not started") .format(controllerId, controller.epoch, replicaId, topicAndPartition, targetState)) try { - replicaState.getOrElseUpdate((topic, partition, replicaId), NonExistentReplica) + replicaState.getOrElseUpdate(partitionAndReplica, NonExistentReplica) val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition) targetState match { case NewReplica => - assertValidPreviousStates(topic, partition, replicaId, List(NonExistentReplica), targetState) + assertValidPreviousStates(partitionAndReplica, List(NonExistentReplica), targetState) // start replica as a follower to the current leader for its partition val leaderIsrAndControllerEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) leaderIsrAndControllerEpochOpt match { @@ -140,22 +161,39 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { replicaAssignment) case None => // new leader request will be sent to this replica when one gets elected } - replicaState.put((topic, partition, replicaId), NewReplica) + replicaState.put(partitionAndReplica, NewReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NewReplica" .format(controllerId, controller.epoch, replicaId, topicAndPartition)) - case NonExistentReplica => - assertValidPreviousStates(topic, partition, replicaId, List(OfflineReplica), targetState) + case ReplicaDeletionStarted => + assertValidPreviousStates(partitionAndReplica, List(OfflineReplica), targetState) + replicaState.put(partitionAndReplica, ReplicaDeletionStarted) // send stop replica command - brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = true) + brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = true, + callbacks.stopReplicaResponseCallback) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionStarted" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + case ReplicaDeletionFailed => + assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionStarted), targetState) + replicaState.put(partitionAndReplica, ReplicaDeletionFailed) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionFailed" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + case ReplicaDeletionSuccessful => + assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionStarted), targetState) + replicaState.put(partitionAndReplica, ReplicaDeletionSuccessful) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionSuccessful" + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + case NonExistentReplica => + assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionSuccessful), targetState) // remove this replica from the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas.filterNot(_ == replicaId)) - replicaState.remove((topic, partition, replicaId)) + replicaState.remove(partitionAndReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NonExistentReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case OnlineReplica => - assertValidPreviousStates(topic, partition, replicaId, List(NewReplica, OnlineReplica, OfflineReplica), targetState) - replicaState((topic, partition, replicaId)) match { + assertValidPreviousStates(partitionAndReplica, + List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionFailed), targetState) + replicaState(partitionAndReplica) match { case NewReplica => // add this replica to the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) @@ -169,17 +207,17 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case Some(leaderIsrAndControllerEpoch) => brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch, replicaAssignment) - replicaState.put((topic, partition, replicaId), OnlineReplica) + replicaState.put(partitionAndReplica, OnlineReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" .format(controllerId, controller.epoch, replicaId, topicAndPartition)) case None => // that means the partition was never in OnlinePartition state, this means the broker never // started a log for that partition and does not have a high watermark value for this partition } - } - replicaState.put((topic, partition, replicaId), OnlineReplica) + replicaState.put(partitionAndReplica, OnlineReplica) case OfflineReplica => - assertValidPreviousStates(topic, partition, replicaId, List(NewReplica, OnlineReplica), targetState) + assertValidPreviousStates(partitionAndReplica, + List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionFailed, ReplicaDeletionSuccessful), targetState) // send stop replica command to the replica so that it stops fetching from the leader brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = false) // As an optimization, the controller removes dead replicas from the ISR @@ -191,7 +229,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { // send the shrunk ISR state change request only to the leader brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) - replicaState.put((topic, partition, replicaId), OfflineReplica) + replicaState.put(partitionAndReplica, OfflineReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" .format(controllerId, controller.epoch, replicaId, topicAndPartition)) false @@ -214,12 +252,34 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } } - private def assertValidPreviousStates(topic: String, partition: Int, replicaId: Int, fromStates: Seq[ReplicaState], + def areAllReplicasForTopicDeleted(topic: String): Boolean = { + val replicasForTopic = controller.controllerContext.replicasForTopic(topic) + val replicaStatesForTopic = replicasForTopic.map(r => (r, replicaState(r))).toMap + debug("Are all replicas for topic %s deleted %s".format(topic, replicaStatesForTopic)) + replicaStatesForTopic.foldLeft(true)((deletionState, r) => deletionState && r._2 == ReplicaDeletionSuccessful) + } + + def isAtLeastOneReplicaInDeletionStartedState(topic: String): Boolean = { + val replicasForTopic = controller.controllerContext.replicasForTopic(topic) + val replicaStatesForTopic = replicasForTopic.map(r => (r, replicaState(r))).toMap + replicaStatesForTopic.foldLeft(false)((deletionState, r) => deletionState || r._2 == ReplicaDeletionStarted) + } + + def replicasInState(topic: String, state: ReplicaState): Set[PartitionAndReplica] = { + replicaState.filter(r => r._1.topic.equals(topic) && r._2 == state).keySet + } + + def replicasInDeletionStates(topic: String): Set[PartitionAndReplica] = { + val deletionStates = Set(ReplicaDeletionStarted, ReplicaDeletionSuccessful, ReplicaDeletionFailed) + replicaState.filter(r => r._1.topic.equals(topic) && deletionStates.contains(r._2)).keySet + } + + private def assertValidPreviousStates(partitionAndReplica: PartitionAndReplica, fromStates: Seq[ReplicaState], targetState: ReplicaState) { - assert(fromStates.contains(replicaState((topic, partition, replicaId))), - "Replica %s for partition [%s,%d] should be in the %s states before moving to %s state" - .format(replicaId, topic, partition, fromStates.mkString(","), targetState) + - ". Instead it is in %s state".format(replicaState((topic, partition, replicaId)))) + assert(fromStates.contains(replicaState(partitionAndReplica)), + "Replica %s should be in the %s states before moving to %s state" + .format(partitionAndReplica, fromStates.mkString(","), targetState) + + ". Instead it is in %s state".format(replicaState(partitionAndReplica))) } private def registerBrokerChangeListener() = { @@ -235,14 +295,23 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { val topic = topicPartition.topic val partition = topicPartition.partition assignedReplicas.foreach { replicaId => + val partitionAndReplica = PartitionAndReplica(topic, partition, replicaId) controllerContext.liveBrokerIds.contains(replicaId) match { - case true => replicaState.put((topic, partition, replicaId), OnlineReplica) - case false => replicaState.put((topic, partition, replicaId), OfflineReplica) + case true => replicaState.put(partitionAndReplica, OnlineReplica) + case false => + // mark replicas on dead brokers as failed for topic deletion, if they belong to a topic to be deleted. + // This is required during controller failover since during controller failover a broker can go down, + // so the replicas on that broker should be moved to ReplicaDeletionFailed to be on the safer side. + replicaState.put(partitionAndReplica, ReplicaDeletionFailed) } } } } + def partitionsAssignedToBroker(topics: Seq[String], brokerId: Int):Seq[TopicAndPartition] = { + controllerContext.partitionReplicaAssignment.filter(_._2.contains(brokerId)).keySet.toSeq + } + /** * This is the zookeeper listener that triggers all the state transitions for a replica */ @@ -250,7 +319,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { this.logIdent = "[BrokerChangeListener on Controller " + controller.config.brokerId + "]: " def handleChildChange(parentPath : String, currentBrokerList : java.util.List[String]) { info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.mkString(","))) - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { if (hasStarted.get) { ControllerStats.leaderElectionTimer.time { try { @@ -282,6 +351,9 @@ sealed trait ReplicaState { def state: Byte } case object NewReplica extends ReplicaState { val state: Byte = 1 } case object OnlineReplica extends ReplicaState { val state: Byte = 2 } case object OfflineReplica extends ReplicaState { val state: Byte = 3 } -case object NonExistentReplica extends ReplicaState { val state: Byte = 4 } +case object ReplicaDeletionStarted extends ReplicaState { val state: Byte = 4} +case object ReplicaDeletionSuccessful extends ReplicaState { val state: Byte = 5} +case object ReplicaDeletionFailed extends ReplicaState { val state: Byte = 6} +case object NonExistentReplica extends ReplicaState { val state: Byte = 7 } diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala new file mode 100644 index 0000000..acfa50f --- /dev/null +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -0,0 +1,371 @@ +/** + * 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 collection.mutable +import kafka.utils.{ShutdownableThread, Logging, ZkUtils} +import kafka.utils.Utils._ +import collection.Set +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.api.{StopReplicaResponse, RequestOrResponse} + +/** + * 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 has a background thread that handles topic deletion. The purpose of having this background thread + * is to accommodate the TTL feature, when we have it. This thread is signaled whenever deletion for a topic needs to + * be started or resumed. Currently, a topic's deletion can be started only by the onPartitionDeletion callback on the + * controller. In the future, it can be triggered based on the configured TTL for the topic. A topic's deletion will + * be halted 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 + * 3.3 preferred replica election for partitions of that topic is in progress + * (though this is not strictly required since it holds the controller lock for the entire duration from start to end) + * 4. Topic deletion is resumed when - + * 4.1 broker hosting one of the replicas for that topic is started + * 4.2 preferred replica election for partitions of that topic completes + * 4.3 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 the onPartitionDeletion callback 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 (deleteTopicStopReplicaCallback() moves replicas from + * TopicDeletionStarted->TopicDeletionSuccessful depending on the error codes in StopReplicaResponse) + * 5.3 TopicDeletionFailed. (deleteTopicStopReplicaCallback() moves replicas from + * TopicDeletionStarted->TopicDeletionSuccessful 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. The delete topic thread marks a topic successfully deleted only if all replicas are in TopicDeletionSuccessful + * state and it starts the topic deletion teardown mode where it 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. + * @param controller + * @param initialTopicsToBeDeleted The topics that are queued up for deletion in zookeeper at the time of controller failover + * @param initialHaltedTopicsForDeletion The topics for which deletion is halted due to any of the conditions mentioned in #3 above + */ +class TopicDeletionManager(controller: KafkaController, + initialTopicsToBeDeleted: Set[String] = Set.empty, + initialHaltedTopicsForDeletion: Set[String] = Set.empty) extends Logging { + val controllerContext = controller.controllerContext + val partitionStateMachine = controller.partitionStateMachine + val replicaStateMachine = controller.replicaStateMachine + var topicsToBeDeleted: mutable.Set[String] = mutable.Set.empty[String] ++ initialTopicsToBeDeleted + var haltedTopicsForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++ + (initialHaltedTopicsForDeletion & initialTopicsToBeDeleted) + var topicDeletionInProgress: mutable.Set[String] = mutable.Set.empty + val deleteTopicsCond = controllerContext.controllerLock.newCondition() + var deleteTopicStateChanged: Boolean = false + var deleteTopicsThread: DeleteTopicsThread = null + + /** + * Invoked at the end of new controller initiation + */ + def start() { + deleteTopicsThread = new DeleteTopicsThread() + deleteTopicStateChanged = true + deleteTopicsThread.start() + } + + /** + * Invoked when the current controller resigns. At this time, all state for topic deletion should be cleared + */ + def shutdown() { + deleteTopicsThread.shutdown() + topicsToBeDeleted.clear() + haltedTopicsForDeletion.clear() + topicDeletionInProgress.clear() + } + + /** + * 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]) { + topicsToBeDeleted ++= topics + resumeTopicDeletionThread() + } + + /** + * 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 + * 3. Preferred replica election completes. Any partitions belonging to topics queued up for deletion finished + * preferred replica election + * @param topics Topics for which deletion can be resumed + */ + def resumeDeletionForTopics(topics: Set[String] = Set.empty) { + val topicsToResumeDeletion = topics & topicsToBeDeleted + if(topicsToResumeDeletion.size > 0) { + haltedTopicsForDeletion --= topicsToResumeDeletion + resumeTopicDeletionThread() + } + } + + /** + * 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 ReplicaDeletionFailed state. Also, the topic is added to the list of topics + * for which deletion is halted until further notice. The delete topic thread is notified so it can retry topic deletion + * if it has received a response for all replicas of a topic to be deleted + * @param replicas Replicas for which deletion has failed + */ + def failReplicaDeletion(replicas: Set[PartitionAndReplica]) { + val replicasThatFailedToDelete = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) + if(replicasThatFailedToDelete.size > 0) { + val topics = replicasThatFailedToDelete.map(_.topic) + debug("Deletion failed for replicas %s. Halting deletion for topics %s" + .format(replicasThatFailedToDelete.mkString(","), topics)) + controller.replicaStateMachine.handleStateChanges(replicasThatFailedToDelete, ReplicaDeletionFailed) + haltTopicDeletion(topics) + resumeTopicDeletionThread() + } + } + + /** + * Halt delete topic if - + * 1. replicas being down + * 2. partition reassignment in progress for some partitions of the topic + * 3. preferred replica election in progress for some partitions of the topic + * @param topics Topics for which deletion should be halted. No op if the topic is was not previously queued up for deletion + */ + def haltTopicDeletion(topics: Set[String]) { + haltedTopicsForDeletion ++= topicsToBeDeleted & topics + } + + def isTopicDeletionHalted(topic: String): Boolean = { + haltedTopicsForDeletion.contains(topic) + } + + def isTopicDeletionInProgress(topic: String): Boolean = { + topicDeletionInProgress.contains(topic) + } + + def isTopicQueuedUpForDeletion(topic: String): Boolean = { + topicsToBeDeleted.contains(topic) + } + + /** + * Invoked by the delete-topic-thread to wait until events that either trigger, restart or halt topic deletion occur. + * controllerLock should be acquired before invoking this API + */ + private def awaitTopicDeletionNotification() { + while(!deleteTopicStateChanged) { + info("Waiting for signal to start or continue topic deletion") + deleteTopicsCond.await() + } + deleteTopicStateChanged = false + } + + /** + * Signals the delete-topic-thread to process topic deletion + */ + private def resumeTopicDeletionThread() { + deleteTopicStateChanged = true + deleteTopicsCond.signal() + } + + /** + * 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. The delete + * topic thread is notified so it can tear down the topic if all replicas of a topic have been successfully deleted + * @param replicas Replicas that were successfully deleted by the broker + */ + private def completeReplicaDeletion(replicas: Set[PartitionAndReplica]) { + val successfullyDeletedReplicas = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) + debug("Deletion successfully completed for replicas %s".format(successfullyDeletedReplicas.mkString(","))) + controller.replicaStateMachine.handleStateChanges(successfullyDeletedReplicas, ReplicaDeletionSuccessful) + resumeTopicDeletionThread() + } + + /** + * Topic deletion can be retried if - + * 1. Topic deletion is currently not in progress for that topic + * 2. Topic deletion is currently halted for that topic + * @param topic Topic + * @return Whether or not deletion can be retried for the topic + */ + private def isTopicEligibleForDeletion(topic: String): Boolean = { + !isTopicDeletionInProgress(topic) && !haltedTopicsForDeletion.contains(topic) + } + + /** + * If the topic is in topicsToBeDeleted, but not in topicDeletionInProgress, then deletion is retried for that topic + * @param topic Topic for which deletion should be retried + */ + private def retryTopicDeletion(topic: String) { + topicDeletionInProgress -= topic + // reset replica states to OfflineReplica + val allReplicas = controller.replicaStateMachine.replicasInDeletionStates(topic) + controller.replicaStateMachine.handleStateChanges(allReplicas, OfflineReplica) + } + + private def completeDeleteTopic(topic: String) { + val replicasForDeletedTopic = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionSuccessful) + // controller will remove this replica from the state machine as well as its partition assignment cache + replicaStateMachine.handleStateChanges(replicasForDeletedTopic, NonExistentReplica) + val partitionsForDeletedTopic = controllerContext.partitionsForTopic(topic) + // move respective partition to OfflinePartition and NonExistentPartition state + partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, OfflinePartition) + partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, NonExistentPartition) + topicDeletionInProgress -= topic + topicsToBeDeleted -= topic + controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) + controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic)) + controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic)) + controllerContext.removeTopic(topic) + } + + /** + * This callback is invoked by the DeleteTopics thread with the list of topics to be deleted + * It invokes the delete partition callback for all partitions of a topic + */ + private def onTopicDeletion(topics: Set[String]) { + info("Topic deletion callback for %s".format(topics.mkString(","))) + val partitionReplicaAssignmentByTopic = controllerContext.partitionReplicaAssignment.groupBy(p => p._1.topic) + topics.foreach { topic => + onPartitionDeletion(partitionReplicaAssignmentByTopic(topic).map(_._1).toSet) + } + } + + /** + * Invoked by the onPartitionDeletion callback. It is the 2nd step of topic deletion, the first being sending + * UpdateMetadata requests to all brokers to start rejecting requests for deleted topics. As part of starting deletion, + * the topics are added to the in progress list. As long as a topic is in the in progress list, deletion for that topic + * is never retried. A topic is removed from the in progress list when + * 1. Either the topic is successfully deleted OR + * 2. No replica for the topic is in ReplicaDeletionStarted state and at least one replica is in ReplicaDeletionFailed state + * If the topic is in topicsToBeDeleted, but not in topicDeletionInProgress, then deletion is retried for that topic + * As part of starting deletion, all replicas are moved to the ReplicaDeletionStarted state where the controller sends + * the replicas a StopReplicaRequest (delete=true) + * @param replicasForTopicsToBeDeleted + */ + private def startReplicaDeletion(replicasForTopicsToBeDeleted: Set[PartitionAndReplica]) { + replicasForTopicsToBeDeleted.groupBy(_.topic).foreach { case(topic, replicas) => + topicDeletionInProgress += topic + // send update metadata so that brokers stop serving data + controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + var aliveReplicasForTopic = controllerContext.allLiveReplicas().filter(p => p.topic.equals(topic)) + val deadReplicasForTopic = replicasForTopicsToBeDeleted -- aliveReplicasForTopic + // move dead replicas directly to failed state + replicaStateMachine.handleStateChanges(deadReplicasForTopic, ReplicaDeletionFailed) + // send stop replica to all followers that are not in the OfflineReplica state so they stop sending fetch requests to the leader + replicaStateMachine.handleStateChanges(aliveReplicasForTopic, OfflineReplica) + debug("Deletion started for replicas %s".format(aliveReplicasForTopic.mkString(","))) + controller.replicaStateMachine.handleStateChanges(aliveReplicasForTopic, ReplicaDeletionStarted, + new Callbacks.CallbackBuilder().stopReplicaCallback(deleteTopicStopReplicaCallback).build) + if(deadReplicasForTopic.size > 0) + haltTopicDeletion(Set(topic)) + } + } + + /** + * This callback is invoked by the delete topic callback with the list of partitions for topics to be deleted + * It does the following - + * 1. Send UpdateMetadataRequest to all live brokers (that are not shutting down) with all partitions except those for + * which the topics are being deleted. The brokers start rejecting all client requests with UnknownTopicOrPartitionException + * 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(partitionsToBeDeleted: Set[TopicAndPartition]) { + info("Partition deletion callback for %s".format(partitionsToBeDeleted.mkString(","))) + val replicasPerPartition = controllerContext.replicasForPartition(partitionsToBeDeleted) + startReplicaDeletion(replicasPerPartition) + } + + private def deleteTopicStopReplicaCallback(stopReplicaResponseObj: RequestOrResponse, replicaId: Int) { + val stopReplicaResponse = stopReplicaResponseObj.asInstanceOf[StopReplicaResponse] + debug("Delete topic callback invoked for %s".format(stopReplicaResponse)) + val partitionsInError = if(stopReplicaResponse.errorCode != ErrorMapping.NoError) { + stopReplicaResponse.responseMap.keySet + } else + stopReplicaResponse.responseMap.filter(p => p._2 != ErrorMapping.NoError).map(_._1).toSet + val replicasInError = partitionsInError.map(p => PartitionAndReplica(p.topic, p.partition, replicaId)) + inLock(controllerContext.controllerLock) { + // move all the failed replicas to ReplicaDeletionFailed + failReplicaDeletion(replicasInError) + if(replicasInError.size != stopReplicaResponse.responseMap.size) { + // some replicas could have been successfully deleted + val deletedReplicas = stopReplicaResponse.responseMap.keySet -- partitionsInError + completeReplicaDeletion(deletedReplicas.map(p => PartitionAndReplica(p.topic, p.partition, replicaId))) + } + } + } + + class DeleteTopicsThread() extends ShutdownableThread("delete-topics-thread") { + val zkClient = controllerContext.zkClient + override def doWork() { + inLock(controllerContext.controllerLock) { + awaitTopicDeletionNotification() + val topicsQueuedForDeletion = Set.empty[String] ++ topicsToBeDeleted + if(topicsQueuedForDeletion.size > 0) + info("Handling deletion for topics " + topicsQueuedForDeletion.mkString(",")) + topicsQueuedForDeletion.foreach { topic => + // check if topic deletion is currently in progress + if(isTopicDeletionInProgress(topic)) { + // if all replicas are marked as deleted successfully, then topic deletion is done + if(controller.replicaStateMachine.areAllReplicasForTopicDeleted(topic)) { + // clear up all state for this topic from controller cache and zookeeper + completeDeleteTopic(topic) + info("Deletion of topic %s successfully completed".format(topic)) + } else { + if(controller.replicaStateMachine.isAtLeastOneReplicaInDeletionStartedState(topic)) { + // ignore since topic deletion is in progress + val replicasInDeletionStartedState = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionStarted) + val replicaIds = replicasInDeletionStartedState.map(_.replica) + val partitions = replicasInDeletionStartedState.map(r => TopicAndPartition(r.topic, r.partition)) + info("Deletion for replicas %s for partition %s of topic %s in progress".format(replicaIds.mkString(","), + partitions.mkString(","), topic)) + } else { + // if you come here, then no replica is in TopicDeletionStarted and all replicas are not in + // TopicDeletionSuccessful. That means, there is at least one failed replica, which means topic deletion + // should be retried + val replicasInTopicDeletionFailedState = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionFailed) + // queue topic for deletion retry + retryTopicDeletion(topic) + info("Retrying delete topic for topic %s since replicas %s were not successfully deleted" + .format(topic, replicasInTopicDeletionFailedState.mkString(","))) + } + } + } + } + // Try delete topic if it is eligible for deletion. + topicsToBeDeleted.foreach { topic => + if(isTopicEligibleForDeletion(topic)) { + info("Deletion of topic %s (re)started".format(topic)) + // topic deletion should be retried + onTopicDeletion(Set(topic)) + } else if(isTopicDeletionHalted(topic)) { + info("Not retrying deletion of topic %s at this time since it is halted".format(topic)) + } + } + } + } + } +} + diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index d22dabd..ab04b3f 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -76,7 +76,7 @@ class BlockingChannel( val host: String, // but let's do it to be sure. swallow(channel.close()) swallow(channel.socket.close()) - swallow(readChannel.close()) + if(readChannel != null) swallow(readChannel.close()) channel = null; readChannel = null; writeChannel = null connected = false } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index bd7940b..c56ad50 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -23,9 +23,7 @@ import kafka.message._ import kafka.network._ import kafka.log._ import kafka.utils.ZKGroupTopicDirs -import org.apache.log4j.Logger import scala.collection._ -import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic._ import kafka.metrics.KafkaMetricsGroup @@ -54,7 +52,7 @@ class KafkaApis(val requestChannel: RequestChannel, private val delayedRequestMetrics = new DelayedRequestMetrics /* following 3 data structures are updated by the update metadata request * and is queried by the topic metadata request. */ - var leaderCache: mutable.Map[TopicAndPartition, PartitionStateInfo] = + var metadataCache: mutable.Map[TopicAndPartition, PartitionStateInfo] = new mutable.HashMap[TopicAndPartition, PartitionStateInfo]() private val aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]() private val partitionMetadataLock = new Object @@ -87,7 +85,16 @@ class KafkaApis(val requestChannel: RequestChannel, request.apiLocalCompleteTimeMs = SystemTime.milliseconds } + // ensureTopicExists is only for client facing requests + private def ensureTopicExists(topic: String) = { + if(!metadataCache.exists { case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic.equals(topic)} ) + throw new UnknownTopicOrPartitionException("Topic " + topic + " either doesn't exist or is in the process of being deleted") + } + def handleLeaderAndIsrRequest(request: RequestChannel.Request) { + // ensureTopicExists is only for client facing requests + // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they + // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] try { val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest) @@ -101,6 +108,9 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleStopReplicaRequest(request: RequestChannel.Request) { + // ensureTopicExists is only for client facing requests + // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they + // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.requestObj.asInstanceOf[StopReplicaRequest] val (response, error) = replicaManager.stopReplicas(stopReplicaRequest) val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.correlationId, response.toMap, error) @@ -110,6 +120,9 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateMetadataRequest(request: RequestChannel.Request) { val updateMetadataRequest = request.requestObj.asInstanceOf[UpdateMetadataRequest] + // ensureTopicExists is only for client facing requests + // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they + // stop serving data to clients for the topic being deleted val stateChangeLogger = replicaManager.stateChangeLogger if(updateMetadataRequest.controllerEpoch < replicaManager.controllerEpoch) { val stateControllerEpochErrorMessage = ("Broker %d received update metadata request with correlation id %d from an " + @@ -124,18 +137,38 @@ class KafkaApis(val requestChannel: RequestChannel, // cache the list of alive brokers in the cluster updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b)) updateMetadataRequest.partitionStateInfos.foreach { partitionState => - leaderCache.put(partitionState._1, partitionState._2) + metadataCache.put(partitionState._1, partitionState._2) if(stateChangeLogger.isTraceEnabled) stateChangeLogger.trace(("Broker %d cached leader info %s for partition %s in response to UpdateMetadata request " + "sent by controller %d epoch %d with correlation id %d").format(brokerId, partitionState._2, partitionState._1, updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) } + // remove the topics that don't exist in the UpdateMetadata request since those are the topics that are + // currently being deleted by the controller + val topicsKnownToThisBroker = metadataCache.map{ + case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet + val topicsKnownToTheController = updateMetadataRequest.partitionStateInfos.map { + case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet + val deletedTopics = topicsKnownToThisBroker -- topicsKnownToTheController + val partitionsToBeDeleted = metadataCache.filter { + case(topicAndPartition, partitionStateInfo) => deletedTopics.contains(topicAndPartition.topic) + }.keySet + partitionsToBeDeleted.foreach { partition => + metadataCache.remove(partition) + if(stateChangeLogger.isTraceEnabled) + stateChangeLogger.trace(("Broker %d deleted partition %s from metadata cache in response to UpdateMetadata request " + + "sent by controller %d epoch %d with correlation id %d").format(brokerId, partition, + updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) + } } val updateMetadataResponse = new UpdateMetadataResponse(updateMetadataRequest.correlationId) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(updateMetadataResponse))) } def handleControlledShutdownRequest(request: RequestChannel.Request) { + // ensureTopicExists is only for client facing requests + // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they + // stop serving data to clients for the topic being deleted val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest] val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId) val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId, @@ -245,6 +278,7 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) try { + ensureTopicExists(topicAndPartition.topic) val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) val info = partitionOpt match { @@ -347,6 +381,7 @@ class KafkaApis(val requestChannel: RequestChannel, case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => val partitionData = try { + ensureTopicExists(topic) val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId) BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(messages.sizeInBytes) @@ -417,6 +452,7 @@ class KafkaApis(val requestChannel: RequestChannel, val responseMap = offsetRequest.requestInfo.map(elem => { val (topicAndPartition, partitionOffsetRequestInfo) = elem try { + ensureTopicExists(topicAndPartition.topic) // ensure leader exists val localReplica = if(!offsetRequest.isFromDebuggingClient) replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) @@ -524,18 +560,18 @@ class KafkaApis(val requestChannel: RequestChannel, metadataRequest.topics.toSet else { partitionMetadataLock synchronized { - leaderCache.keySet.map(_.topic) + metadataCache.keySet.map(_.topic) } } } val topicMetadataList = partitionMetadataLock synchronized { uniqueTopics.map { topic => - if(leaderCache.keySet.map(_.topic).contains(topic)) { - val partitionStateInfo = leaderCache.filter(p => p._1.topic.equals(topic)) + if(metadataCache.keySet.map(_.topic).contains(topic)) { + val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic)) val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition) val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) => - val replicas = leaderCache(topicAndPartition).allReplicas + val replicas = metadataCache(topicAndPartition).allReplicas var replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq var leaderInfo: Option[Broker] = None var isrInfo: Seq[Broker] = Nil @@ -607,6 +643,7 @@ class KafkaApis(val requestChannel: RequestChannel, case (topicAndPartition, metaAndError) => { val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) try { + ensureTopicExists(topicAndPartition.topic) if(metaAndError.metadata != null && metaAndError.metadata.length > config.offsetMetadataMaxSize) { (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) } else { @@ -632,6 +669,7 @@ class KafkaApis(val requestChannel: RequestChannel, val responseInfo = offsetFetchRequest.requestInfo.map( t => { val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, t.topic) try { + ensureTopicExists(t.topic) val payloadOpt = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir + "/" + t.partition)._1 payloadOpt match { case Some(payload) => { diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 9dca55c..8c69d09 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -20,7 +20,6 @@ package kafka.server import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} -import kafka.common._ import java.net.InetAddress diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala index b5719f8..19f61a9 100644 --- a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala +++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala @@ -19,7 +19,6 @@ package kafka.server import scala.collection._ import kafka.utils.Logging import kafka.common._ -import java.util.concurrent.locks.ReentrantLock import java.io._ /** diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f9d10d3..21bba48 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -114,7 +114,8 @@ class ReplicaManager(val config: KafkaConfig, } def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = { - stateChangeLogger.trace("Broker %d handling stop replica for partition [%s,%d]".format(localBrokerId, topic, partitionId)) + stateChangeLogger.trace("Broker %d handling stop replica (delete=%s) for partition [%s,%d]".format(localBrokerId, + deletePartition.toString, topic, partitionId)) val errorCode = ErrorMapping.NoError getPartition(topic, partitionId) match { case Some(partition) => @@ -126,15 +127,18 @@ class ReplicaManager(val config: KafkaConfig, if (removedPartition != null) removedPartition.delete() // this will delete the local log } - case None => //do nothing if replica no longer exists + case None => //do nothing if replica no longer exists. This can happen during delete topic retries + stateChangeLogger.trace("Broker %d ignoring stop replica (delete=%s) for partition [%s,%d] as replica doesn't exist on broker" + .format(localBrokerId, deletePartition, topic, partitionId)) } - stateChangeLogger.trace("Broker %d finished handling stop replica for partition [%s,%d]".format(localBrokerId, topic, partitionId)) + stateChangeLogger.trace("Broker %d finished handling stop replica (delete=%s) for partition [%s,%d]" + .format(localBrokerId, deletePartition, topic, partitionId)) errorCode } - def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[(String, Int), Short], Short) = { + def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[TopicAndPartition, Short], Short) = { replicaStateChangeLock synchronized { - val responseMap = new collection.mutable.HashMap[(String, Int), Short] + val responseMap = new collection.mutable.HashMap[TopicAndPartition, Short] if(stopReplicaRequest.controllerEpoch < controllerEpoch) { stateChangeLogger.warn("Broker %d received stop replica request from an old controller epoch %d." .format(localBrokerId, stopReplicaRequest.controllerEpoch) + @@ -142,14 +146,11 @@ class ReplicaManager(val config: KafkaConfig, (responseMap, ErrorMapping.StaleControllerEpochCode) } else { controllerEpoch = stopReplicaRequest.controllerEpoch - val responseMap = new HashMap[(String, Int), Short] // First stop fetchers for all partitions, then stop the corresponding replicas - replicaFetcherManager.removeFetcherForPartitions(stopReplicaRequest.partitions.map { - case (topic, partition) => TopicAndPartition(topic, partition) - }) - for((topic, partitionId) <- stopReplicaRequest.partitions){ - val errorCode = stopReplica(topic, partitionId, stopReplicaRequest.deletePartitions) - responseMap.put((topic, partitionId), errorCode) + replicaFetcherManager.removeFetcherForPartitions(stopReplicaRequest.partitions.map(r => TopicAndPartition(r.topic, r.partition))) + for(topicAndPartition <- stopReplicaRequest.partitions){ + val errorCode = stopReplica(topicAndPartition.topic, topicAndPartition.partition, stopReplicaRequest.deletePartitions) + responseMap.put(topicAndPartition, errorCode) } (responseMap, ErrorMapping.NoError) } @@ -252,10 +253,10 @@ class ReplicaManager(val config: KafkaConfig, val partitionsTobeLeader = partitionState .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId} - val partitionsTobeFollower = (partitionState -- partitionsTobeLeader.keys) + val partitionsToBeFollower = (partitionState -- partitionsTobeLeader.keys) if (!partitionsTobeLeader.isEmpty) makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap) - if (!partitionsTobeFollower.isEmpty) makeFollowers(controllerId, controllerEpoch, partitionsTobeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) + if (!partitionsToBeFollower.isEmpty) makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions // have been completely populated before starting the checkpointing there by avoiding weird race conditions diff --git a/core/src/main/scala/kafka/server/TopicConfigManager.scala b/core/src/main/scala/kafka/server/TopicConfigManager.scala index 42e98dd..d41fd33 100644 --- a/core/src/main/scala/kafka/server/TopicConfigManager.scala +++ b/core/src/main/scala/kafka/server/TopicConfigManager.scala @@ -105,8 +105,13 @@ class TopicConfigManager(private val zkClient: ZkClient, log.config = logConfig lastExecutedChange = changeId info("Processed topic config change %d for topic %s, setting new config to %s.".format(changeId, topic, props)) - } else if (now - stat.getCtime > changeExpirationMs) { - /* this change is now obsolete, try to delete it unless it is the last change left */ + } else { + if (now - stat.getCtime > changeExpirationMs) { + /* this change is now obsolete, try to delete it unless it is the last change left */ + error("Ignoring topic config change %d for topic %s since the change has expired") + } else { + error("Ignoring topic config change %d for topic %s since the topic may have been deleted") + } ZkUtils.deletePath(zkClient, changeZnode) } } diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala index b189619..e5b6ff1 100644 --- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala +++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala @@ -17,12 +17,12 @@ package kafka.server import kafka.utils.ZkUtils._ -import kafka.utils.{Json, Utils, SystemTime, Logging} +import kafka.utils.Utils._ +import kafka.utils.{Json, SystemTime, Logging} import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.I0Itec.zkclient.IZkDataListener import kafka.controller.ControllerContext import kafka.controller.KafkaController -import kafka.common.KafkaException /** * This class handles zookeeper based leader election based on an ephemeral path. The election module does not handle @@ -44,7 +44,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, val leaderChangeListener = new LeaderChangeListener def startup { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { controllerContext.zkClient.subscribeDataChanges(electionPath, leaderChangeListener) elect } @@ -102,7 +102,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, */ @throws(classOf[Exception]) def handleDataChange(dataPath: String, data: Object) { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { leaderId = KafkaController.parseControllerId(data.toString) info("New leader is %d".format(leaderId)) } @@ -115,7 +115,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, */ @throws(classOf[Exception]) def handleDataDeleted(dataPath: String) { - controllerContext.controllerLock synchronized { + inLock(controllerContext.controllerLock) { debug("%s leader change listener fired for path %s to handle data deleted: trying to elect as a leader" .format(brokerId, dataPath)) if(amILeader) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index b42e52b..fa86bb9 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -47,20 +47,24 @@ object ZkUtils extends Logging { val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" + val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" - def getTopicPath(topic: String): String ={ + def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic } - def getTopicPartitionsPath(topic: String): String ={ + def getTopicPartitionsPath(topic: String): String = { getTopicPath(topic) + "/partitions" } def getTopicConfigPath(topic: String): String = TopicConfigPath + "/" + topic - - def getController(zkClient: ZkClient): Int= { + + def getDeleteTopicPath(topic: String): String = + DeleteTopicsPath + "/" + topic + + def getController(zkClient: ZkClient): Int = { readDataMaybeNull(zkClient, ControllerPath)._1 match { case Some(controller) => KafkaController.parseControllerId(controller) case None => throw new KafkaException("Controller doesn't exist") diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 59de1b4..d5644ea 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -27,6 +27,7 @@ import kafka.utils.{Logging, ZkUtils, TestUtils} import kafka.common.{TopicExistsException, TopicAndPartition} import kafka.server.{KafkaServer, KafkaConfig} import java.io.File +import TestUtils._ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { @@ -161,9 +162,9 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { }, 1000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) // in sync replicas should not have any replica that is not in the new assigned replicas - checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) + checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas) - ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -190,8 +191,8 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { }, 1000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas) - checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) - ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) + ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -218,8 +219,8 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { }, 2000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas) - checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) - ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) + ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -255,12 +256,12 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { reassignPartitionsCommand.reassignPartitions // create brokers val servers = TestUtils.createBrokerConfigs(2).map(b => TestUtils.createServer(new KafkaConfig(b))) - TestUtils.waitUntilTrue(checkIfReassignPartitionPathExists, 1000) + TestUtils.waitUntilTrue(() => checkIfReassignPartitionPathExists(zkClient), 1000) val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned) assertEquals("Partition should have been reassigned to 0, 1", newReplicas, assignedReplicas) - checkForPhantomInSyncReplicas(topic, partitionToBeReassigned, assignedReplicas) + checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas) // ensure that there are no under replicated partitions - ensureNoUnderReplicatedPartitions(topic, partitionToBeReassigned, assignedReplicas, servers) + ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers) assertTrue(TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet, 5000)) servers.foreach(_.shutdown()) } @@ -319,9 +320,9 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { try { // wait for the update metadata request to trickle to the brokers assertTrue("Topic test not created after timeout", TestUtils.waitUntilTrue(() => - activeServers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), 1000)) + activeServers.foldLeft(true)(_ && _.apis.metadataCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.isr.size != 3), 1000)) assertEquals(0, partitionsRemaining.size) - var partitionStateInfo = activeServers.head.apis.leaderCache(TopicAndPartition(topic, partition)) + var partitionStateInfo = activeServers.head.apis.metadataCache(TopicAndPartition(topic, partition)) var leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader assertEquals(0, leaderAfterShutdown) assertEquals(2, partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.isr.size) @@ -330,15 +331,15 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { partitionsRemaining = controller.shutdownBroker(1) assertEquals(0, partitionsRemaining.size) activeServers = servers.filter(s => s.config.brokerId == 0) - partitionStateInfo = activeServers.head.apis.leaderCache(TopicAndPartition(topic, partition)) + partitionStateInfo = activeServers.head.apis.metadataCache(TopicAndPartition(topic, partition)) leaderAfterShutdown = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader assertEquals(0, leaderAfterShutdown) - assertTrue(servers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) + assertTrue(servers.foldLeft(true)(_ && _.apis.metadataCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) partitionsRemaining = controller.shutdownBroker(0) assertEquals(1, partitionsRemaining.size) // leader doesn't change since all the replicas are shut down - assertTrue(servers.foldLeft(true)(_ && _.apis.leaderCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) + assertTrue(servers.foldLeft(true)(_ && _.apis.metadataCache(TopicAndPartition(topic, partition)).leaderIsrAndControllerEpoch.leaderAndIsr.leader == 0)) } finally { servers.foreach(_.shutdown()) @@ -389,27 +390,4 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { } } - private def checkForPhantomInSyncReplicas(topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int]) { - val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) - // in sync replicas should not have any replica that is not in the new assigned replicas - val phantomInSyncReplicas = inSyncReplicas.toSet -- assignedReplicas.toSet - assertTrue("All in sync replicas %s must be in the assigned replica list %s".format(inSyncReplicas, assignedReplicas), - phantomInSyncReplicas.size == 0) - } - - private def ensureNoUnderReplicatedPartitions(topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int], - servers: Seq[KafkaServer]) { - val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) - assertFalse("Reassigned partition [%s,%d] is underreplicated".format(topic, partitionToBeReassigned), - inSyncReplicas.size < assignedReplicas.size) - val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionToBeReassigned) - assertTrue("Reassigned partition [%s,%d] is unavailable".format(topic, partitionToBeReassigned), leader.isDefined) - val leaderBroker = servers.filter(s => s.config.brokerId == leader.get).head - assertTrue("Reassigned partition [%s,%d] is underreplicated as reported by the leader %d".format(topic, partitionToBeReassigned, leader.get), - leaderBroker.replicaManager.underReplicatedPartitionCount() == 0) - } - - private def checkIfReassignPartitionPathExists(): Boolean = { - ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath) - } } diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala new file mode 100644 index 0000000..e0d3b92 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -0,0 +1,377 @@ +package kafka.admin + +import org.scalatest.junit.JUnit3Suite +import kafka.zk.ZooKeeperTestHarness +import junit.framework.Assert._ +import kafka.utils.{ZkUtils, TestUtils} +import kafka.server.{KafkaServer, KafkaConfig} +import org.junit.Test +import kafka.common._ +import kafka.producer.{ProducerConfig, Producer} +import java.util.Properties +import kafka.api._ +import kafka.consumer.SimpleConsumer +import kafka.producer.KeyedMessage +import kafka.common.TopicAndPartition +import kafka.api.PartitionOffsetRequestInfo + +class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { + + @Test + def testDeleteTopicWithAllAliveReplicas() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + verifyTopicDeletion(topic, servers) + servers.foreach(_.shutdown()) + } + + @Test + def testResumeDeleteTopicWithRecoveredFollower() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + // shut down one follower replica + val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last + follower.shutdown() + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // check if all replicas but the one that is shut down has deleted the log + assertTrue("Replicas 0,1 have not deleted log in 1000ms", TestUtils.waitUntilTrue(() => + servers.dropRight(1).foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty), 1000)) + // ensure topic deletion is halted + assertTrue("Admin path /admin/delete_topic/test path deleted in 1000ms even when a follower replica is down", + TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 500)) + // restart follower replica + follower.startup() + verifyTopicDeletion(topic, servers) + servers.foreach(_.shutdown()) + } + + @Test + def testResumeDeleteTopicOnControllerFailover() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // shut down the controller to trigger controller failover during delete topic + val controllerId = ZkUtils.getController(zkClient) + val controller = servers.filter(s => s.config.brokerId == controllerId).head + controller.shutdown() + // ensure topic deletion is halted + assertTrue("Admin path /admin/delete_topic/test path deleted in 500ms even when a replica is down", + TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 500)) + // restart follower replica + controller.startup() + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + assertTrue("Admin path /admin/delete_topic/test path not deleted in 4000ms even after a follower replica is restarted", + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 4000)) + assertTrue("Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted", + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), 100)) + // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper + assertTrue("Replica logs not deleted after delete topic is complete", + servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty)) + servers.foreach(_.shutdown()) + } + + @Test + def testRequestHandlingDuringDeleteTopic() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // shut down one follower replica + var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last + follower.shutdown() + // test if produce requests are failed with UnknownTopicOrPartitionException during delete topic + val props1 = new Properties() + props1.put("metadata.broker.list", servers.map(s => s.config.hostName + ":" + s.config.port).mkString(",")) + props1.put("serializer.class", "kafka.serializer.StringEncoder") + props1.put("request.required.acks", "1") + val producerConfig1 = new ProducerConfig(props1) + val producer1 = new Producer[String, String](producerConfig1) + try{ + producer1.send(new KeyedMessage[String, String](topic, "test", "test1")) + fail("Test should fail because the topic is being deleted") + } catch { + case e: FailedToSendMessageException => + case oe: Throwable => fail("fails with exception", oe) + } finally { + producer1.close() + } + // test if fetch requests fail during delete topic + servers.filter(s => s.config.brokerId != follower.config.brokerId).foreach { server => + val consumer = new SimpleConsumer(server.config.hostName, server.config.port, 1000000, 64*1024, "") + val request = new FetchRequestBuilder() + .clientId("test-client") + .addFetch(topic, 0, 0, 10000) + .build() + val fetched = consumer.fetch(request) + val fetchResponse = fetched.data(topicAndPartition) + assertTrue("Fetch should fail with UnknownTopicOrPartitionCode", fetchResponse.error == ErrorMapping.UnknownTopicOrPartitionCode) + } + // test if offset requests fail during delete topic + servers.filter(s => s.config.brokerId != follower.config.brokerId).foreach { server => + val consumer = new SimpleConsumer(server.config.hostName, server.config.port, 1000000, 64*1024, "") + val offsetRequest = new OffsetRequest(Map(topicAndPartition -> new PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) + val offsetResponse = consumer.getOffsetsBefore(offsetRequest) + val errorCode = offsetResponse.partitionErrorAndOffsets(topicAndPartition).error + assertTrue("Offset request should fail with UnknownTopicOrPartitionCode", errorCode == ErrorMapping.UnknownTopicOrPartitionCode) + // test if offset fetch requests fail during delete topic + val offsetFetchRequest = new OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val offsetFetchResponse = consumer.fetchOffsets(offsetFetchRequest) + val offsetFetchErrorCode = offsetFetchResponse.requestInfo(topicAndPartition).error + assertTrue("Offset fetch request should fail with UnknownTopicOrPartitionCode", + offsetFetchErrorCode == ErrorMapping.UnknownTopicOrPartitionCode) + // TODO: test if offset commit requests fail during delete topic + } + // restart follower replica + follower.startup() + verifyTopicDeletion(topic, servers) + servers.foreach(_.shutdown()) + } + + @Test + def testPreferredReplicaElectionDuringDeleteTopic() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + // shut down the controller to move the leader to a non preferred replica before delete topic + val preferredReplicaId = 0 + val preferredReplica = servers.filter(s => s.config.brokerId == preferredReplicaId).head + preferredReplica.shutdown() + preferredReplica.startup() + val newLeaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 3000, leaderIdOpt) + assertTrue("New leader should be elected prior to delete topic", newLeaderIdOpt.isDefined) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // test preferred replica election + val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkClient, Set(topicAndPartition)) + preferredReplicaElection.moveLeaderToPreferredReplica() + val leaderAfterPreferredReplicaElectionOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000, newLeaderIdOpt) + assertTrue("Preferred replica election should not move leader during delete topic", + leaderAfterPreferredReplicaElectionOpt.isEmpty || leaderAfterPreferredReplicaElectionOpt.get == newLeaderIdOpt.get) + val newControllerId = ZkUtils.getController(zkClient) + val newController = servers.filter(s => s.config.brokerId == newControllerId).head + assertFalse("Preferred replica election should fail", + newController.kafkaController.controllerContext.partitionsUndergoingPreferredReplicaElection.contains(topicAndPartition)) + verifyTopicDeletion(topic, servers) + servers.foreach(_.shutdown()) + } + + @Test + def testDeleteTopicDuringPreferredReplicaElection() { + val topic = "test" + val topicAndPartition = TopicAndPartition(topic, 0) + val servers = createTestTopicAndCluster(topic) + var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + // shut down the controller to move the leader to a non preferred replica before delete topic + val preferredReplicaId = 0 + val preferredReplica = servers.filter(s => s.config.brokerId == preferredReplicaId).head + preferredReplica.shutdown() + preferredReplica.startup() + val newLeaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 3000, leaderIdOpt) + assertTrue("New leader should be elected prior to delete topic", newLeaderIdOpt.isDefined) + // test preferred replica election + val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkClient, Set(topicAndPartition)) + preferredReplicaElection.moveLeaderToPreferredReplica() + // start topic deletion during preferred replica election. This should halt topic deletion but eventually + // complete it successfully + AdminUtils.deleteTopic(zkClient, topic) + val newControllerId = ZkUtils.getController(zkClient) + val newController = servers.filter(s => s.config.brokerId == newControllerId).head + assertTrue("Preferred replica election should succeed after 1000ms", TestUtils.waitUntilTrue(() => + !newController.kafkaController.controllerContext.partitionsUndergoingPreferredReplicaElection.contains(topicAndPartition), 1000)) + verifyTopicDeletion(topic, servers) + servers.foreach(_.shutdown()) + } + + @Test + def testPartitionReassignmentDuringDeleteTopic() { + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + val topic = "test" + val topicAndPartition = TopicAndPartition(topic, 0) + // create brokers + val allServers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) + // create the topic + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + // wait until replica log is created on every broker + assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => + res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // start partition reassignment at the same time right after delete topic. In this case, reassignment will fail since + // the topic is being deleted + // reassign partition 0 + val oldAssignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) + val newReplicas = Seq(1, 2, 3) + val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) + assertTrue("Partition reassignment should fail for [test,0]", reassignPartitionsCommand.reassignPartitions()) + // wait until reassignment is completed + TestUtils.waitUntilTrue(() => { + val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); + ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, + Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentFailed; + }, 1000) + val controllerId = ZkUtils.getController(zkClient) + val controller = servers.filter(s => s.config.brokerId == controllerId).head + assertFalse("Partition reassignment should fail", + controller.kafkaController.controllerContext.partitionsBeingReassigned.contains(topicAndPartition)) + val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) + assertEquals("Partition should not be reassigned to 0, 1, 2", oldAssignedReplicas, assignedReplicas) + verifyTopicDeletion(topic, servers) + allServers.foreach(_.shutdown()) + } + + @Test + def testDeleteTopicDuringPartitionReassignment() { + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + val topic = "test" + val topicAndPartition = TopicAndPartition(topic, 0) + // create brokers + val allServers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) + // create the topic + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + // wait until replica log is created on every broker + assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => + res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + var leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0) + assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined) + // start partition reassignment at the same time right before delete topic. In this case, reassignment will succeed + // reassign partition 0 + val newReplicas = Seq(1, 2, 3) + val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) + assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions()) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // wait until reassignment is completed + TestUtils.waitUntilTrue(() => { + val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas); + ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas, + Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted; + }, 1000) + val controllerId = ZkUtils.getController(zkClient) + val controller = servers.filter(s => s.config.brokerId == controllerId).head + assertFalse("Partition reassignment should complete", + controller.kafkaController.controllerContext.partitionsBeingReassigned.contains(topicAndPartition)) + val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) + assertEquals("Partition should be reassigned to 1,2,3", newReplicas, assignedReplicas) + verifyTopicDeletion(topic, allServers) + allServers.foreach(_.shutdown()) + } + + @Test + def testDeleteTopicDuringAddPartition() { + val topic = "test" + val servers = createTestTopicAndCluster(topic) + // add partitions to topic + val topicAndPartition = TopicAndPartition(topic, 0) + val newPartition = TopicAndPartition(topic, 1) + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // test if topic deletion is resumed + verifyTopicDeletion(topic, servers) + // verify that new partition doesn't exist on any broker either + assertTrue("Replica logs not for new partition [test,1] not deleted after delete topic is complete", TestUtils.waitUntilTrue(() => + servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty), 1000)) + servers.foreach(_.shutdown()) + } + + @Test + def testAddPartitionDuringDeleteTopic() { + val topic = "test" + val topicAndPartition = TopicAndPartition(topic, 0) + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + // add partitions to topic + val newPartition = TopicAndPartition(topic, 1) + AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") + verifyTopicDeletion(topic, servers) + // verify that new partition doesn't exist on any broker either + assertTrue("Replica logs not deleted after delete topic is complete", + servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty)) + servers.foreach(_.shutdown()) + } + + @Test + def testRecreateTopicAfterDeletion() { + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + val topic = "test" + val topicAndPartition = TopicAndPartition(topic, 0) + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + verifyTopicDeletion(topic, servers) + // re-create topic on same replicas + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + // wait until leader is elected + val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined) + // check if all replica logs are created + assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => + res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + servers.foreach(_.shutdown()) + } + + @Test + def testTopicConfigChangesDuringDeleteTopic() { + val topic = "test" + val topicAndPartition = TopicAndPartition(topic, 0) + val servers = createTestTopicAndCluster(topic) + val topicConfigs = new Properties() + topicConfigs.put("segment.ms", "1000000") + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + verifyTopicDeletion(topic, servers) + // make topic config changes + try { + AdminUtils.changeTopicConfig(zkClient, topic, topicConfigs) + fail("Should fail with AdminOperationException for topic doesn't exist") + } catch { + case e: AdminOperationException => // expected + } + servers.foreach(_.shutdown()) + } + + private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { + val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) + val topicAndPartition = TopicAndPartition(topic, 0) + // create brokers + val servers = TestUtils.createBrokerConfigs(3).map(b => TestUtils.createServer(new KafkaConfig(b))) + // create the topic + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + // wait until replica log is created on every broker + assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => + res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + servers + } + + private def verifyTopicDeletion(topic: String, servers: Seq[KafkaServer]) { + val topicAndPartition = TopicAndPartition(topic, 0) + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + assertTrue("Admin path /admin/delete_topic/test path not deleted in 1000ms even after a replica is restarted", + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), 1000)) + assertTrue("Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted", + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), 100)) + // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper + assertTrue("Replica logs not deleted after delete topic is complete", + servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty)) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 8df0982..1ce26df 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -25,7 +25,7 @@ import kafka.message.{Message, ByteBufferMessageSet} import kafka.cluster.Broker import collection.mutable._ import kafka.common.{TopicAndPartition, ErrorMapping, OffsetMetadataAndError} -import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.controller.{PartitionAndReplica, LeaderIsrAndControllerEpoch} object SerializationTestUtils{ @@ -100,12 +100,12 @@ object SerializationTestUtils{ def createTestStopReplicaRequest() : StopReplicaRequest = { new StopReplicaRequest(controllerId = 0, controllerEpoch = 1, correlationId = 0, deletePartitions = true, - partitions = collection.immutable.Set((topic1, 0), (topic2, 0))) + partitions = collection.immutable.Set(TopicAndPartition(topic1, 0),TopicAndPartition(topic2, 0))) } def createTestStopReplicaResponse() : StopReplicaResponse = { - val responseMap = Map(((topic1, 0), ErrorMapping.NoError), - ((topic2, 0), ErrorMapping.NoError)) + val responseMap = Map((TopicAndPartition(topic1, 0), ErrorMapping.NoError), + (TopicAndPartition(topic2, 0), ErrorMapping.NoError)) new StopReplicaResponse(0, responseMap.toMap) } diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 9aea67b..e102285 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -85,7 +85,9 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.createTopic(zkClient, topic, 1, 1) val logManager = server.getLogManager - val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig) + assertTrue("Log for partition [topic,0] should be created", + waitUntilTrue(() => logManager.getLog(TopicAndPartition(topic, part)).isDefined, 1000)) + val log = logManager.getLog(TopicAndPartition(topic, part)).get val message = new Message(Integer.toString(42).getBytes()) for(i <- 0 until 20) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index c0475d0..6a96d80 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -30,6 +30,7 @@ import kafka.api.{OffsetCommitRequest, OffsetFetchRequest} import kafka.utils.TestUtils._ import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError} import scala.util.Random +import kafka.admin.AdminUtils class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val random: Random = new Random() @@ -66,6 +67,11 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // Commit an offset val topicAndPartition = TopicAndPartition(topic, 0) + val expectedReplicaAssignment = Map(0 -> List(1)) + // create the topic + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) + val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(offset=42L))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) @@ -105,8 +111,23 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topic3 = "topic-3" val topic4 = "topic-4" + val expectedReplicaAssignment = Map(0 -> List(1)) + // create the topic + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, expectedReplicaAssignment) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, expectedReplicaAssignment) + var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) + leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) + leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) + leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic4, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) + val commitRequest = OffsetCommitRequest("test-group", Map( - TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="metadata one"), + TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="metadata one"), TopicAndPartition(topic2, 0) -> OffsetMetadataAndError(offset=43L, metadata="metadata two"), TopicAndPartition(topic3, 0) -> OffsetMetadataAndError(offset=44L, metadata="metadata three"), TopicAndPartition(topic2, 1) -> OffsetMetadataAndError(offset=45L) @@ -152,6 +173,11 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testLargeMetadataPayload() { val topicAndPartition = TopicAndPartition("large-metadata", 0) + val expectedReplicaAssignment = Map(0 -> List(1)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topicAndPartition.topic, expectedReplicaAssignment) + var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) + val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( offset=42L, metadata=random.nextString(server.config.offsetMetadataMaxSize) @@ -173,6 +199,10 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testNullMetadata() { val topicAndPartition = TopicAndPartition("null-metadata", 0) + val expectedReplicaAssignment = Map(0 -> List(1)) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topicAndPartition.topic, expectedReplicaAssignment) + var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000) + assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( offset=42L, metadata=null diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 03e6266..35723dc 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -28,6 +28,7 @@ import kafka.api._ import scala.Some import org.junit.Assert._ import kafka.common.TopicAndPartition +import kafka.controller.LeaderIsrAndControllerEpoch class SimpleFetchTest extends JUnit3Suite { @@ -94,7 +95,9 @@ class SimpleFetchTest extends JUnit3Suite { // don't provide replica or leader callbacks since they will not be tested here val requestChannel = new RequestChannel(2, 5) val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) - + val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) + apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo) + EasyMock.replay(partitionStateInfo) // This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log val goodFetch = new FetchRequestBuilder() .replicaId(Request.OrdinaryConsumerId) @@ -163,6 +166,9 @@ class SimpleFetchTest extends JUnit3Suite { val requestChannel = new RequestChannel(2, 5) val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) + val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) + apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo) + EasyMock.replay(partitionStateInfo) /** * This fetch, coming from a replica, requests all data at offset "15". Because the request is coming diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 426b1a7..500eeca 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -513,7 +513,7 @@ object TestUtils extends Logging { def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long) = { Assert.assertTrue("Partition [%s,%d] metadata not propagated after timeout".format(topic, partition), TestUtils.waitUntilTrue(() => - servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition))), timeout)) + servers.foldLeft(true)(_ && _.apis.metadataCache.keySet.contains(TopicAndPartition(topic, partition))), timeout)) } def writeNonsenseToFile(fileName: File, position: Long, size: Int) { @@ -530,7 +530,30 @@ object TestUtils extends Logging { file.write(random.nextInt(255)) file.close() } - + + def checkForPhantomInSyncReplicas(zkClient: ZkClient, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int]) { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) + // in sync replicas should not have any replica that is not in the new assigned replicas + val phantomInSyncReplicas = inSyncReplicas.toSet -- assignedReplicas.toSet + assertTrue("All in sync replicas %s must be in the assigned replica list %s".format(inSyncReplicas, assignedReplicas), + phantomInSyncReplicas.size == 0) + } + + def ensureNoUnderReplicatedPartitions(zkClient: ZkClient, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int], + servers: Seq[KafkaServer]) { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned) + assertFalse("Reassigned partition [%s,%d] is underreplicated".format(topic, partitionToBeReassigned), + inSyncReplicas.size < assignedReplicas.size) + val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionToBeReassigned) + assertTrue("Reassigned partition [%s,%d] is unavailable".format(topic, partitionToBeReassigned), leader.isDefined) + val leaderBroker = servers.filter(s => s.config.brokerId == leader.get).head + assertTrue("Reassigned partition [%s,%d] is underreplicated as reported by the leader %d".format(topic, partitionToBeReassigned, leader.get), + leaderBroker.replicaManager.underReplicatedPartitionCount() == 0) + } + + def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = { + ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath) + } } object TestZKUtils {