You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jj...@apache.org on 2014/04/18 23:03:58 UTC

git commit: KAFKA-1355; Avoid sending all topic metadata on state changes. Reviewed by Neha Narkhede, Timothy Chen and Guozhang Wang.

Repository: kafka
Updated Branches:
  refs/heads/0.8.1 7502696e1 -> b18d2c379


KAFKA-1355; Avoid sending all topic metadata on state changes. Reviewed
by Neha Narkhede, Timothy Chen and Guozhang Wang.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b18d2c37
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b18d2c37
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b18d2c37

Branch: refs/heads/0.8.1
Commit: b18d2c379bbaf65629c041f75918a4711e4063d6
Parents: 7502696
Author: Joel Koshy <jj...@gmail.com>
Authored: Thu Apr 17 14:46:08 2014 -0700
Committer: Joel Koshy <jj...@gmail.com>
Committed: Fri Apr 18 14:03:21 2014 -0700

----------------------------------------------------------------------
 .../scala/kafka/api/LeaderAndIsrRequest.scala   |  2 +
 .../controller/ControllerChannelManager.scala   | 55 +++++++++++---------
 .../kafka/controller/KafkaController.scala      |  8 +--
 .../kafka/controller/TopicDeletionManager.scala | 17 +++---
 .../src/main/scala/kafka/server/KafkaApis.scala | 30 ++++++-----
 5 files changed, 64 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/b18d2c37/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
index 0311737..3e40817 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
@@ -32,6 +32,8 @@ import collection.Set
 object LeaderAndIsr {
   val initialLeaderEpoch: Int = 0
   val initialZKVersion: Int = 0
+  val NoLeader = -1
+  val LeaderDuringDelete = -2
 }
 
 case class LeaderAndIsr(var leader: Int, var leaderEpoch: Int, var isr: List[Int], var zkVersion: Int) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/b18d2c37/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index f17d976..f79c1dc 100644
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -211,7 +211,8 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
       leaderAndIsrRequestMap(brokerId).put((topic, partition),
         PartitionStateInfo(leaderIsrAndControllerEpoch, replicas.toSet))
     }
-    addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq)
+    addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq,
+                                       Set(TopicAndPartition(topic, partition)))
   }
 
   def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, deletePartition: Boolean,
@@ -232,34 +233,40 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
    *
    */
   def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int],
+                                         partitions: collection.Set[TopicAndPartition] = Set.empty[TopicAndPartition],
                                          callback: (RequestOrResponse) => Unit = null) {
-    val partitionList = controllerContext.partitionLeadershipInfo.keySet.dropWhile(
-      p => controller.deleteTopicManager.isTopicQueuedUpForDeletion(p.topic))
-    if(partitionList.size > 0) {
-      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.filter(b => b >= 0).foreach { brokerId =>
-              updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo])
-              updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo)
-            }
-          case None =>
-            info("Leader not assigned yet for partition %s. Skip sending udpate metadata request".format(partition))
-        }
-      }
-    } else {
-      if(controllerContext.partitionLeadershipInfo.keySet.size > 0) {
-        // last set of topics are being deleted
-        controllerContext.partitionLeadershipInfo.foreach { case(partition, leaderIsrAndControllerEpoch) =>
+    def updateMetadataRequestMapFor(partition: TopicAndPartition, beingDeleted: Boolean) {
+      val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition)
+      leaderIsrAndControllerEpochOpt match {
+        case Some(leaderIsrAndControllerEpoch) =>
+          val replicas = controllerContext.partitionReplicaAssignment(partition).toSet
+          val partitionStateInfo = if (beingDeleted) {
+            val leaderAndIsr = new LeaderAndIsr(LeaderAndIsr.LeaderDuringDelete, leaderIsrAndControllerEpoch.leaderAndIsr.isr)
+            PartitionStateInfo(LeaderIsrAndControllerEpoch(leaderAndIsr, leaderIsrAndControllerEpoch.controllerEpoch), replicas)
+          } else {
+            PartitionStateInfo(leaderIsrAndControllerEpoch, replicas)
+          }
           brokerIds.filter(b => b >= 0).foreach { brokerId =>
-            updateMetadataRequestMap.put(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo])
+            updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo])
+            updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo)
           }
-        }
+        case None =>
+          info("Leader not yet assigned for partition %s. Skip sending UpdateMetadataRequest.".format(partition))
       }
     }
+
+    val filteredPartitions = {
+      val givenPartitions = if (partitions.isEmpty)
+        controllerContext.partitionLeadershipInfo.keySet
+      else
+        partitions
+      if (controller.deleteTopicManager.partitionsToBeDeleted.isEmpty)
+        givenPartitions
+      else
+        givenPartitions -- controller.deleteTopicManager.partitionsToBeDeleted
+    }
+    filteredPartitions.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = false))
+    controller.deleteTopicManager.partitionsToBeDeleted.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = true))
   }
 
   def sendRequestsToBrokers(controllerEpoch: Int, correlationId: Int) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/b18d2c37/core/src/main/scala/kafka/controller/KafkaController.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index f221f84..21fb715 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -554,7 +554,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
         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)
+        sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicAndPartition))
         // signal delete topic thread if reassignment for some partitions belonging to topics being deleted just completed
         deleteTopicManager.resumeDeletionForTopics(Set(topicAndPartition.topic))
     }
@@ -935,9 +935,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
    * metadata requests
    * @param brokers The brokers that the update metadata request should be sent to
    */
-  def sendUpdateMetadataRequest(brokers: Seq[Int]) {
+  def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) {
     brokerRequestBatch.newBatch()
-    brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers)
+    brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions)
     brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement)
   }
 
@@ -969,7 +969,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg
               "controller was elected with epoch %d. Aborting state change by this controller".format(controllerEpoch))
           if (leaderAndIsr.isr.contains(replicaId)) {
             // if the replica to be removed from the ISR is also the leader, set the new leader value to -1
-            val newLeader = if(replicaId == leaderAndIsr.leader) -1 else leaderAndIsr.leader
+            val newLeader = if (replicaId == leaderAndIsr.leader) LeaderAndIsr.NoLeader else leaderAndIsr.leader
             val newLeaderAndIsr = new LeaderAndIsr(newLeader, leaderAndIsr.leaderEpoch + 1,
               leaderAndIsr.isr.filter(b => b != replicaId), leaderAndIsr.zkVersion + 1)
             // update the new leadership decision in zookeeper or retry

http://git-wip-us.apache.org/repos/asf/kafka/blob/b18d2c37/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
index 40c4c57..c995ec0 100644
--- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
+++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
@@ -72,12 +72,13 @@ class TopicDeletionManager(controller: KafkaController,
   val controllerContext = controller.controllerContext
   val partitionStateMachine = controller.partitionStateMachine
   val replicaStateMachine = controller.replicaStateMachine
-  var topicsToBeDeleted: mutable.Set[String] = mutable.Set.empty[String] ++ initialTopicsToBeDeleted
+  val topicsToBeDeleted: mutable.Set[String] = mutable.Set.empty[String] ++ initialTopicsToBeDeleted
+  val partitionsToBeDeleted: mutable.Set[TopicAndPartition] = topicsToBeDeleted.flatMap(controllerContext.partitionsForTopic)
   val deleteLock = new ReentrantLock()
-  var topicsIneligibleForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++
+  val topicsIneligibleForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++
     (initialTopicsIneligibleForDeletion & initialTopicsToBeDeleted)
   val deleteTopicsCond = deleteLock.newCondition()
-  var deleteTopicStateChanged: AtomicBoolean = new AtomicBoolean(false)
+  val deleteTopicStateChanged: AtomicBoolean = new AtomicBoolean(false)
   var deleteTopicsThread: DeleteTopicsThread = null
   val isDeleteTopicEnabled = controller.config.deleteTopicEnable
 
@@ -99,6 +100,7 @@ class TopicDeletionManager(controller: KafkaController,
     if(isDeleteTopicEnabled) {
       deleteTopicsThread.shutdown()
       topicsToBeDeleted.clear()
+      partitionsToBeDeleted.clear()
       topicsIneligibleForDeletion.clear()
     }
   }
@@ -112,6 +114,7 @@ class TopicDeletionManager(controller: KafkaController,
   def enqueueTopicsForDeletion(topics: Set[String]) {
     if(isDeleteTopicEnabled) {
       topicsToBeDeleted ++= topics
+      partitionsToBeDeleted ++= topics.flatMap(controllerContext.partitionsForTopic)
       resumeTopicDeletionThread()
     }
   }
@@ -264,6 +267,7 @@ class TopicDeletionManager(controller: KafkaController,
     partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, OfflinePartition)
     partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, NonExistentPartition)
     topicsToBeDeleted -= topic
+    partitionsToBeDeleted.retain(_.topic != topic)
     controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic))
     controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicConfigPath(topic))
     controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic))
@@ -277,7 +281,8 @@ class TopicDeletionManager(controller: KafkaController,
   private def onTopicDeletion(topics: Set[String]) {
     info("Topic deletion callback for %s".format(topics.mkString(",")))
     // send update metadata so that brokers stop serving data for topics to be deleted
-    controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq)
+    val partitions = topics.flatMap(controllerContext.partitionsForTopic)
+    controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, partitions)
     val partitionReplicaAssignmentByTopic = controllerContext.partitionReplicaAssignment.groupBy(p => p._1.topic)
     topics.foreach { topic =>
       onPartitionDeletion(partitionReplicaAssignmentByTopic(topic).map(_._1).toSet)
@@ -322,8 +327,8 @@ class TopicDeletionManager(controller: KafkaController,
   /**
    * 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
+   * 1. Send UpdateMetadataRequest to all live brokers (that are not shutting down) for partitions that 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

http://git-wip-us.apache.org/repos/asf/kafka/blob/b18d2c37/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 0513e45..9e569eb 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -138,23 +138,25 @@ class KafkaApis(val requestChannel: RequestChannel,
                     stateChangeLogger: StateChangeLogger) {
       inLock(partitionMetadataLock.writeLock()) {
         updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b))
+        val topicsToDelete = mutable.Set[String]()
         updateMetadataRequest.partitionStateInfos.foreach { partitionState =>
-          addPartitionInfoInternal(partitionState._1.topic, partitionState._1.partition, partitionState._2)
-          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))
+          if (partitionState._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader == LeaderAndIsr.LeaderDuringDelete) {
+            topicsToDelete.add(partitionState._1.topic)
+          } else {
+            addPartitionInfoInternal(partitionState._1.topic, partitionState._1.partition, partitionState._2)
+            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 = cache.keySet
-        val topicsKnownToTheController = updateMetadataRequest.partitionStateInfos.map {
-          case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet
-        val deletedTopics = topicsKnownToThisBroker -- topicsKnownToTheController
-        deletedTopics.foreach { topic =>
+
+        topicsToDelete.foreach { topic =>
           cache.remove(topic)
-          stateChangeLogger.trace(("Broker %d deleted partitions for topic %s from metadata cache in response to UpdateMetadata request " +
-            "sent by controller %d epoch %d with correlation id %d").format(brokerId, topic,
-              updateMetadataRequest.controllerId, updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId))
+          stateChangeLogger.trace(("Broker %d deleted partitions for topic %s from metadata cache in response to " +
+                                   "UpdateMetadata request  sent by controller %d epoch %d with correlation id %d")
+                                   .format(brokerId, topic, updateMetadataRequest.controllerId,
+                                           updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId))
         }
       }
     }