mirror of https://github.com/apache/kafka.git
				
				
				
			MINOR: Move ZkMetadataCache into its own file. (#10942)
Reviewers: Jason Gustafson <jason@confluent.io>
This commit is contained in:
		
							parent
							
								
									d9b898b678
								
							
						
					
					
						commit
						9f71db96fd
					
				|  | @ -17,31 +17,13 @@ | ||||||
| 
 | 
 | ||||||
| package kafka.server | package kafka.server | ||||||
| 
 | 
 | ||||||
| import java.util |  | ||||||
| import java.util.Collections |  | ||||||
| import java.util.concurrent.locks.ReentrantReadWriteLock |  | ||||||
| 
 |  | ||||||
| import kafka.admin.BrokerMetadata | import kafka.admin.BrokerMetadata | ||||||
| 
 | 
 | ||||||
| import scala.collection.{Seq, Set, mutable} |  | ||||||
| import scala.jdk.CollectionConverters._ |  | ||||||
| import kafka.cluster.{Broker, EndPoint} |  | ||||||
| import kafka.api._ |  | ||||||
| import kafka.controller.StateChangeLogger |  | ||||||
| import kafka.server.metadata.RaftMetadataCache | import kafka.server.metadata.RaftMetadataCache | ||||||
| import kafka.utils.CoreUtils._ | import org.apache.kafka.common.{Cluster, Node, TopicPartition} | ||||||
| import kafka.utils.Logging |  | ||||||
| import kafka.utils.Implicits._ |  | ||||||
| import org.apache.kafka.common.internals.Topic |  | ||||||
| import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState |  | ||||||
| import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} |  | ||||||
| import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic |  | ||||||
| import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition |  | ||||||
| import org.apache.kafka.common.message.{MetadataResponseData, UpdateMetadataRequestData} | import org.apache.kafka.common.message.{MetadataResponseData, UpdateMetadataRequestData} | ||||||
| import org.apache.kafka.common.network.ListenerName | import org.apache.kafka.common.network.ListenerName | ||||||
| import org.apache.kafka.common.protocol.Errors | import org.apache.kafka.common.requests.UpdateMetadataRequest | ||||||
| import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} |  | ||||||
| import org.apache.kafka.common.security.auth.SecurityProtocol |  | ||||||
| 
 | 
 | ||||||
| trait MetadataCache { | trait MetadataCache { | ||||||
| 
 | 
 | ||||||
|  | @ -118,367 +100,3 @@ object MetadataCache { | ||||||
|     new RaftMetadataCache(brokerId) |     new RaftMetadataCache(brokerId) | ||||||
|   } |   } | ||||||
| } | } | ||||||
| 
 |  | ||||||
| /** |  | ||||||
|  *  A cache for the state (e.g., current leader) of each partition. This cache is updated through |  | ||||||
|  *  UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. |  | ||||||
|  */ |  | ||||||
| class ZkMetadataCache(brokerId: Int) extends MetadataCache with Logging { |  | ||||||
| 
 |  | ||||||
|   private val partitionMetadataLock = new ReentrantReadWriteLock() |  | ||||||
|   //this is the cache state. every MetadataSnapshot instance is immutable, and updates (performed under a lock) |  | ||||||
|   //replace the value with a completely new one. this means reads (which are not under any lock) need to grab |  | ||||||
|   //the value of this var (into a val) ONCE and retain that read copy for the duration of their operation. |  | ||||||
|   //multiple reads of this value risk getting different snapshots. |  | ||||||
|   @volatile private var metadataSnapshot: MetadataSnapshot = MetadataSnapshot(partitionStates = mutable.AnyRefMap.empty, |  | ||||||
|     topicIds = Map.empty, controllerId = None, aliveBrokers = mutable.LongMap.empty, aliveNodes = mutable.LongMap.empty) |  | ||||||
| 
 |  | ||||||
|   this.logIdent = s"[MetadataCache brokerId=$brokerId] " |  | ||||||
|   private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None) |  | ||||||
| 
 |  | ||||||
|   // This method is the main hotspot when it comes to the performance of metadata requests, |  | ||||||
|   // we should be careful about adding additional logic here. Relatedly, `brokers` is |  | ||||||
|   // `List[Integer]` instead of `List[Int]` to avoid a collection copy. |  | ||||||
|   // filterUnavailableEndpoints exists to support v0 MetadataResponses |  | ||||||
|   private def maybeFilterAliveReplicas(snapshot: MetadataSnapshot, |  | ||||||
|                                        brokers: java.util.List[Integer], |  | ||||||
|                                        listenerName: ListenerName, |  | ||||||
|                                        filterUnavailableEndpoints: Boolean): java.util.List[Integer] = { |  | ||||||
|     if (!filterUnavailableEndpoints) { |  | ||||||
|       brokers |  | ||||||
|     } else { |  | ||||||
|       val res = new util.ArrayList[Integer](math.min(snapshot.aliveBrokers.size, brokers.size)) |  | ||||||
|       for (brokerId <- brokers.asScala) { |  | ||||||
|         if (hasAliveEndpoint(snapshot, brokerId, listenerName)) |  | ||||||
|           res.add(brokerId) |  | ||||||
|       } |  | ||||||
|       res |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   // errorUnavailableEndpoints exists to support v0 MetadataResponses |  | ||||||
|   // If errorUnavailableListeners=true, return LISTENER_NOT_FOUND if listener is missing on the broker. |  | ||||||
|   // Otherwise, return LEADER_NOT_AVAILABLE for broker unavailable and missing listener (Metadata response v5 and below). |  | ||||||
|   private def getPartitionMetadata(snapshot: MetadataSnapshot, topic: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean, |  | ||||||
|                                    errorUnavailableListeners: Boolean): Option[Iterable[MetadataResponsePartition]] = { |  | ||||||
|     snapshot.partitionStates.get(topic).map { partitions => |  | ||||||
|       partitions.map { case (partitionId, partitionState) => |  | ||||||
|         val topicPartition = new TopicPartition(topic, partitionId.toInt) |  | ||||||
|         val leaderBrokerId = partitionState.leader |  | ||||||
|         val leaderEpoch = partitionState.leaderEpoch |  | ||||||
|         val maybeLeader = getAliveEndpoint(snapshot, leaderBrokerId, listenerName) |  | ||||||
| 
 |  | ||||||
|         val replicas = partitionState.replicas |  | ||||||
|         val filteredReplicas = maybeFilterAliveReplicas(snapshot, replicas, listenerName, errorUnavailableEndpoints) |  | ||||||
| 
 |  | ||||||
|         val isr = partitionState.isr |  | ||||||
|         val filteredIsr = maybeFilterAliveReplicas(snapshot, isr, listenerName, errorUnavailableEndpoints) |  | ||||||
| 
 |  | ||||||
|         val offlineReplicas = partitionState.offlineReplicas |  | ||||||
| 
 |  | ||||||
|         maybeLeader match { |  | ||||||
|           case None => |  | ||||||
|             val error = if (!snapshot.aliveBrokers.contains(leaderBrokerId)) { // we are already holding the read lock |  | ||||||
|               debug(s"Error while fetching metadata for $topicPartition: leader not available") |  | ||||||
|               Errors.LEADER_NOT_AVAILABLE |  | ||||||
|             } else { |  | ||||||
|               debug(s"Error while fetching metadata for $topicPartition: listener $listenerName " + |  | ||||||
|                 s"not found on leader $leaderBrokerId") |  | ||||||
|               if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE |  | ||||||
|             } |  | ||||||
| 
 |  | ||||||
|             new MetadataResponsePartition() |  | ||||||
|               .setErrorCode(error.code) |  | ||||||
|               .setPartitionIndex(partitionId.toInt) |  | ||||||
|               .setLeaderId(MetadataResponse.NO_LEADER_ID) |  | ||||||
|               .setLeaderEpoch(leaderEpoch) |  | ||||||
|               .setReplicaNodes(filteredReplicas) |  | ||||||
|               .setIsrNodes(filteredIsr) |  | ||||||
|               .setOfflineReplicas(offlineReplicas) |  | ||||||
| 
 |  | ||||||
|           case Some(_) => |  | ||||||
|             val error = if (filteredReplicas.size < replicas.size) { |  | ||||||
|               debug(s"Error while fetching metadata for $topicPartition: replica information not available for " + |  | ||||||
|                 s"following brokers ${replicas.asScala.filterNot(filteredReplicas.contains).mkString(",")}") |  | ||||||
|               Errors.REPLICA_NOT_AVAILABLE |  | ||||||
|             } else if (filteredIsr.size < isr.size) { |  | ||||||
|               debug(s"Error while fetching metadata for $topicPartition: in sync replica information not available for " + |  | ||||||
|                 s"following brokers ${isr.asScala.filterNot(filteredIsr.contains).mkString(",")}") |  | ||||||
|               Errors.REPLICA_NOT_AVAILABLE |  | ||||||
|             } else { |  | ||||||
|               Errors.NONE |  | ||||||
|             } |  | ||||||
| 
 |  | ||||||
|             new MetadataResponsePartition() |  | ||||||
|               .setErrorCode(error.code) |  | ||||||
|               .setPartitionIndex(partitionId.toInt) |  | ||||||
|               .setLeaderId(maybeLeader.map(_.id()).getOrElse(MetadataResponse.NO_LEADER_ID)) |  | ||||||
|               .setLeaderEpoch(leaderEpoch) |  | ||||||
|               .setReplicaNodes(filteredReplicas) |  | ||||||
|               .setIsrNodes(filteredIsr) |  | ||||||
|               .setOfflineReplicas(offlineReplicas) |  | ||||||
|         } |  | ||||||
|       } |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   /** |  | ||||||
|    * Check whether a broker is alive and has a registered listener matching the provided name. |  | ||||||
|    * This method was added to avoid unnecessary allocations in [[maybeFilterAliveReplicas]], which is |  | ||||||
|    * a hotspot in metadata handling. |  | ||||||
|    */ |  | ||||||
|   private def hasAliveEndpoint(snapshot: MetadataSnapshot, brokerId: Int, listenerName: ListenerName): Boolean = { |  | ||||||
|     snapshot.aliveNodes.get(brokerId).exists(_.contains(listenerName)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   /** |  | ||||||
|    * Get the endpoint matching the provided listener if the broker is alive. Note that listeners can |  | ||||||
|    * be added dynamically, so a broker with a missing listener could be a transient error. |  | ||||||
|    * |  | ||||||
|    * @return None if broker is not alive or if the broker does not have a listener named `listenerName`. |  | ||||||
|    */ |  | ||||||
|   private def getAliveEndpoint(snapshot: MetadataSnapshot, brokerId: Int, listenerName: ListenerName): Option[Node] = { |  | ||||||
|     snapshot.aliveNodes.get(brokerId).flatMap(_.get(listenerName)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   // errorUnavailableEndpoints exists to support v0 MetadataResponses |  | ||||||
|   def getTopicMetadata(topics: Set[String], |  | ||||||
|                        listenerName: ListenerName, |  | ||||||
|                        errorUnavailableEndpoints: Boolean = false, |  | ||||||
|                        errorUnavailableListeners: Boolean = false): Seq[MetadataResponseTopic] = { |  | ||||||
|     val snapshot = metadataSnapshot |  | ||||||
|     topics.toSeq.flatMap { topic => |  | ||||||
|       getPartitionMetadata(snapshot, topic, listenerName, errorUnavailableEndpoints, errorUnavailableListeners).map { partitionMetadata => |  | ||||||
|         new MetadataResponseTopic() |  | ||||||
|           .setErrorCode(Errors.NONE.code) |  | ||||||
|           .setName(topic) |  | ||||||
|           .setTopicId(snapshot.topicIds.getOrElse(topic, Uuid.ZERO_UUID)) |  | ||||||
|           .setIsInternal(Topic.isInternal(topic)) |  | ||||||
|           .setPartitions(partitionMetadata.toBuffer.asJava) |  | ||||||
|       } |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   override def getAllTopics(): Set[String] = { |  | ||||||
|     getAllTopics(metadataSnapshot) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   override def getTopicPartitions(topicName: String): Set[TopicPartition] = { |  | ||||||
|     metadataSnapshot.partitionStates.getOrElse(topicName, Map.empty).values. |  | ||||||
|       map(p => new TopicPartition(topicName, p.partitionIndex())).toSet |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   private def getAllTopics(snapshot: MetadataSnapshot): Set[String] = { |  | ||||||
|     snapshot.partitionStates.keySet |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   private def getAllPartitions(snapshot: MetadataSnapshot): Map[TopicPartition, UpdateMetadataPartitionState] = { |  | ||||||
|     snapshot.partitionStates.flatMap { case (topic, partitionStates) => |  | ||||||
|       partitionStates.map { case (partition, state ) => (new TopicPartition(topic, partition.toInt), state) } |  | ||||||
|     }.toMap |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def getNonExistingTopics(topics: Set[String]): Set[String] = { |  | ||||||
|     topics.diff(metadataSnapshot.partitionStates.keySet) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   override def hasAliveBroker(brokerId: Int): Boolean = metadataSnapshot.aliveBrokers.contains(brokerId) |  | ||||||
| 
 |  | ||||||
|   override def getAliveBrokers(): Iterable[BrokerMetadata] = { |  | ||||||
|     metadataSnapshot.aliveBrokers.values.map(b => new BrokerMetadata(b.id, b.rack)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { |  | ||||||
|     metadataSnapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   override def getAliveBrokerNodes(listenerName: ListenerName): Iterable[Node] = { |  | ||||||
|     metadataSnapshot.aliveBrokers.values.flatMap(_.getNode(listenerName)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], |  | ||||||
|                                        topic: String, |  | ||||||
|                                        partitionId: Int, |  | ||||||
|                                        stateInfo: UpdateMetadataPartitionState): Unit = { |  | ||||||
|     val infos = partitionStates.getOrElseUpdate(topic, mutable.LongMap.empty) |  | ||||||
|     infos(partitionId) = stateInfo |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { |  | ||||||
|     metadataSnapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def numPartitions(topic: String): Option[Int] = { |  | ||||||
|     metadataSnapshot.partitionStates.get(topic).map(_.size) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   // if the leader is not known, return None; |  | ||||||
|   // if the leader is known and corresponding node is available, return Some(node) |  | ||||||
|   // if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE) |  | ||||||
|   def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] = { |  | ||||||
|     val snapshot = metadataSnapshot |  | ||||||
|     snapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) map { partitionInfo => |  | ||||||
|       val leaderId = partitionInfo.leader |  | ||||||
| 
 |  | ||||||
|       snapshot.aliveNodes.get(leaderId) match { |  | ||||||
|         case Some(nodeMap) => |  | ||||||
|           nodeMap.getOrElse(listenerName, Node.noNode) |  | ||||||
|         case None => |  | ||||||
|           Node.noNode |  | ||||||
|       } |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = { |  | ||||||
|     val snapshot = metadataSnapshot |  | ||||||
|     snapshot.partitionStates.get(tp.topic).flatMap(_.get(tp.partition)).map { partitionInfo => |  | ||||||
|       val replicaIds = partitionInfo.replicas |  | ||||||
|       replicaIds.asScala |  | ||||||
|         .map(replicaId => replicaId.intValue() -> { |  | ||||||
|           snapshot.aliveBrokers.get(replicaId.longValue()) match { |  | ||||||
|             case Some(broker) => |  | ||||||
|               broker.getNode(listenerName).getOrElse(Node.noNode()) |  | ||||||
|             case None => |  | ||||||
|               Node.noNode() |  | ||||||
|           }}).toMap |  | ||||||
|         .filter(pair => pair match { |  | ||||||
|           case (_, node) => !node.isEmpty |  | ||||||
|         }) |  | ||||||
|     }.getOrElse(Map.empty[Int, Node]) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def getControllerId: Option[Int] = metadataSnapshot.controllerId |  | ||||||
| 
 |  | ||||||
|   def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster = { |  | ||||||
|     val snapshot = metadataSnapshot |  | ||||||
|     val nodes = snapshot.aliveNodes.flatMap { case (id, nodesByListener) => |  | ||||||
|       nodesByListener.get(listenerName).map { node => |  | ||||||
|         id -> node |  | ||||||
|       } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     def node(id: Integer): Node = { |  | ||||||
|       nodes.getOrElse(id.toLong, new Node(id, "", -1)) |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     val partitions = getAllPartitions(snapshot) |  | ||||||
|       .filter { case (_, state) => state.leader != LeaderAndIsr.LeaderDuringDelete } |  | ||||||
|       .map { case (tp, state) => |  | ||||||
|         new PartitionInfo(tp.topic, tp.partition, node(state.leader), |  | ||||||
|           state.replicas.asScala.map(node).toArray, |  | ||||||
|           state.isr.asScala.map(node).toArray, |  | ||||||
|           state.offlineReplicas.asScala.map(node).toArray) |  | ||||||
|       } |  | ||||||
|     val unauthorizedTopics = Collections.emptySet[String] |  | ||||||
|     val internalTopics = getAllTopics(snapshot).filter(Topic.isInternal).asJava |  | ||||||
|     new Cluster(clusterId, nodes.values.toBuffer.asJava, |  | ||||||
|       partitions.toBuffer.asJava, |  | ||||||
|       unauthorizedTopics, internalTopics, |  | ||||||
|       snapshot.controllerId.map(id => node(id)).orNull) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   // This method returns the deleted TopicPartitions received from UpdateMetadataRequest |  | ||||||
|   def updateMetadata(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest): Seq[TopicPartition] = { |  | ||||||
|     inWriteLock(partitionMetadataLock) { |  | ||||||
| 
 |  | ||||||
|       val aliveBrokers = new mutable.LongMap[Broker](metadataSnapshot.aliveBrokers.size) |  | ||||||
|       val aliveNodes = new mutable.LongMap[collection.Map[ListenerName, Node]](metadataSnapshot.aliveNodes.size) |  | ||||||
|       val controllerIdOpt = updateMetadataRequest.controllerId match { |  | ||||||
|         case id if id < 0 => None |  | ||||||
|         case id => Some(id) |  | ||||||
|       } |  | ||||||
| 
 |  | ||||||
|       updateMetadataRequest.liveBrokers.forEach { broker => |  | ||||||
|         // `aliveNodes` is a hot path for metadata requests for large clusters, so we use java.util.HashMap which |  | ||||||
|         // is a bit faster than scala.collection.mutable.HashMap. When we drop support for Scala 2.10, we could |  | ||||||
|         // move to `AnyRefMap`, which has comparable performance. |  | ||||||
|         val nodes = new java.util.HashMap[ListenerName, Node] |  | ||||||
|         val endPoints = new mutable.ArrayBuffer[EndPoint] |  | ||||||
|         broker.endpoints.forEach { ep => |  | ||||||
|           val listenerName = new ListenerName(ep.listener) |  | ||||||
|           endPoints += new EndPoint(ep.host, ep.port, listenerName, SecurityProtocol.forId(ep.securityProtocol)) |  | ||||||
|           nodes.put(listenerName, new Node(broker.id, ep.host, ep.port)) |  | ||||||
|         } |  | ||||||
|         aliveBrokers(broker.id) = Broker(broker.id, endPoints, Option(broker.rack)) |  | ||||||
|         aliveNodes(broker.id) = nodes.asScala |  | ||||||
|       } |  | ||||||
|       aliveNodes.get(brokerId).foreach { listenerMap => |  | ||||||
|         val listeners = listenerMap.keySet |  | ||||||
|         if (!aliveNodes.values.forall(_.keySet == listeners)) |  | ||||||
|           error(s"Listeners are not identical across brokers: $aliveNodes") |  | ||||||
|       } |  | ||||||
| 
 |  | ||||||
|       val newTopicIds = updateMetadataRequest.topicStates().asScala |  | ||||||
|         .map(topicState => (topicState.topicName(), topicState.topicId())) |  | ||||||
|         .filter(_._2 != Uuid.ZERO_UUID).toMap |  | ||||||
|       val topicIds = mutable.Map.empty[String, Uuid] |  | ||||||
|       topicIds ++= metadataSnapshot.topicIds |  | ||||||
|       topicIds ++= newTopicIds |  | ||||||
| 
 |  | ||||||
|       val deletedPartitions = new mutable.ArrayBuffer[TopicPartition] |  | ||||||
|       if (!updateMetadataRequest.partitionStates.iterator.hasNext) { |  | ||||||
|         metadataSnapshot = MetadataSnapshot(metadataSnapshot.partitionStates, topicIds.toMap, controllerIdOpt, aliveBrokers, aliveNodes) |  | ||||||
|       } else { |  | ||||||
|         //since kafka may do partial metadata updates, we start by copying the previous state |  | ||||||
|         val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]](metadataSnapshot.partitionStates.size) |  | ||||||
|         metadataSnapshot.partitionStates.forKeyValue { (topic, oldPartitionStates) => |  | ||||||
|           val copy = new mutable.LongMap[UpdateMetadataPartitionState](oldPartitionStates.size) |  | ||||||
|           copy ++= oldPartitionStates |  | ||||||
|           partitionStates(topic) = copy |  | ||||||
|         } |  | ||||||
| 
 |  | ||||||
|         val traceEnabled = stateChangeLogger.isTraceEnabled |  | ||||||
|         val controllerId = updateMetadataRequest.controllerId |  | ||||||
|         val controllerEpoch = updateMetadataRequest.controllerEpoch |  | ||||||
|         val newStates = updateMetadataRequest.partitionStates.asScala |  | ||||||
|         newStates.foreach { state => |  | ||||||
|           // per-partition logging here can be very expensive due going through all partitions in the cluster |  | ||||||
|           val tp = new TopicPartition(state.topicName, state.partitionIndex) |  | ||||||
|           if (state.leader == LeaderAndIsr.LeaderDuringDelete) { |  | ||||||
|             removePartitionInfo(partitionStates, topicIds, tp.topic, tp.partition) |  | ||||||
|             if (traceEnabled) |  | ||||||
|               stateChangeLogger.trace(s"Deleted partition $tp from metadata cache in response to UpdateMetadata " + |  | ||||||
|                 s"request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") |  | ||||||
|             deletedPartitions += tp |  | ||||||
|           } else { |  | ||||||
|             addOrUpdatePartitionInfo(partitionStates, tp.topic, tp.partition, state) |  | ||||||
|             if (traceEnabled) |  | ||||||
|               stateChangeLogger.trace(s"Cached leader info $state for partition $tp in response to " + |  | ||||||
|                 s"UpdateMetadata request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") |  | ||||||
|           } |  | ||||||
|         } |  | ||||||
|         val cachedPartitionsCount = newStates.size - deletedPartitions.size |  | ||||||
|         stateChangeLogger.info(s"Add $cachedPartitionsCount partitions and deleted ${deletedPartitions.size} partitions from metadata cache " + |  | ||||||
|           s"in response to UpdateMetadata request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") |  | ||||||
| 
 |  | ||||||
|         metadataSnapshot = MetadataSnapshot(partitionStates, topicIds.toMap, controllerIdOpt, aliveBrokers, aliveNodes) |  | ||||||
|       } |  | ||||||
|       deletedPartitions |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def contains(topic: String): Boolean = { |  | ||||||
|     metadataSnapshot.partitionStates.contains(topic) |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   def contains(tp: TopicPartition): Boolean = getPartitionInfo(tp.topic, tp.partition).isDefined |  | ||||||
| 
 |  | ||||||
|   private def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], |  | ||||||
|                                   topicIds: mutable.Map[String, Uuid], topic: String, partitionId: Int): Boolean = { |  | ||||||
|     partitionStates.get(topic).exists { infos => |  | ||||||
|       infos.remove(partitionId) |  | ||||||
|       if (infos.isEmpty) { |  | ||||||
|         partitionStates.remove(topic) |  | ||||||
|         topicIds.remove(topic) |  | ||||||
|       } |  | ||||||
|       true |  | ||||||
|     } |  | ||||||
|   } |  | ||||||
| 
 |  | ||||||
|   case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], |  | ||||||
|                               topicIds: Map[String, Uuid], |  | ||||||
|                               controllerId: Option[Int], |  | ||||||
|                               aliveBrokers: mutable.LongMap[Broker], |  | ||||||
|                               aliveNodes: mutable.LongMap[collection.Map[ListenerName, Node]]) |  | ||||||
| 
 |  | ||||||
| } |  | ||||||
|  |  | ||||||
|  | @ -0,0 +1,406 @@ | ||||||
|  | /** | ||||||
|  |  * 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.server | ||||||
|  | 
 | ||||||
|  | import java.util | ||||||
|  | import java.util.Collections | ||||||
|  | import java.util.concurrent.locks.ReentrantReadWriteLock | ||||||
|  | 
 | ||||||
|  | import kafka.admin.BrokerMetadata | ||||||
|  | 
 | ||||||
|  | import scala.collection.{Seq, Set, mutable} | ||||||
|  | import scala.jdk.CollectionConverters._ | ||||||
|  | import kafka.cluster.{Broker, EndPoint} | ||||||
|  | import kafka.api._ | ||||||
|  | import kafka.controller.StateChangeLogger | ||||||
|  | import kafka.utils.CoreUtils._ | ||||||
|  | import kafka.utils.Logging | ||||||
|  | import kafka.utils.Implicits._ | ||||||
|  | import org.apache.kafka.common.internals.Topic | ||||||
|  | import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState | ||||||
|  | import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} | ||||||
|  | import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic | ||||||
|  | import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition | ||||||
|  | import org.apache.kafka.common.network.ListenerName | ||||||
|  | import org.apache.kafka.common.protocol.Errors | ||||||
|  | import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} | ||||||
|  | import org.apache.kafka.common.security.auth.SecurityProtocol | ||||||
|  | 
 | ||||||
|  | /** | ||||||
|  |  *  A cache for the state (e.g., current leader) of each partition. This cache is updated through | ||||||
|  |  *  UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. | ||||||
|  |  */ | ||||||
|  | class ZkMetadataCache(brokerId: Int) extends MetadataCache with Logging { | ||||||
|  | 
 | ||||||
|  |   private val partitionMetadataLock = new ReentrantReadWriteLock() | ||||||
|  |   //this is the cache state. every MetadataSnapshot instance is immutable, and updates (performed under a lock) | ||||||
|  |   //replace the value with a completely new one. this means reads (which are not under any lock) need to grab | ||||||
|  |   //the value of this var (into a val) ONCE and retain that read copy for the duration of their operation. | ||||||
|  |   //multiple reads of this value risk getting different snapshots. | ||||||
|  |   @volatile private var metadataSnapshot: MetadataSnapshot = MetadataSnapshot(partitionStates = mutable.AnyRefMap.empty, | ||||||
|  |     topicIds = Map.empty, controllerId = None, aliveBrokers = mutable.LongMap.empty, aliveNodes = mutable.LongMap.empty) | ||||||
|  | 
 | ||||||
|  |   this.logIdent = s"[MetadataCache brokerId=$brokerId] " | ||||||
|  |   private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None) | ||||||
|  | 
 | ||||||
|  |   // This method is the main hotspot when it comes to the performance of metadata requests, | ||||||
|  |   // we should be careful about adding additional logic here. Relatedly, `brokers` is | ||||||
|  |   // `List[Integer]` instead of `List[Int]` to avoid a collection copy. | ||||||
|  |   // filterUnavailableEndpoints exists to support v0 MetadataResponses | ||||||
|  |   private def maybeFilterAliveReplicas(snapshot: MetadataSnapshot, | ||||||
|  |                                        brokers: java.util.List[Integer], | ||||||
|  |                                        listenerName: ListenerName, | ||||||
|  |                                        filterUnavailableEndpoints: Boolean): java.util.List[Integer] = { | ||||||
|  |     if (!filterUnavailableEndpoints) { | ||||||
|  |       brokers | ||||||
|  |     } else { | ||||||
|  |       val res = new util.ArrayList[Integer](math.min(snapshot.aliveBrokers.size, brokers.size)) | ||||||
|  |       for (brokerId <- brokers.asScala) { | ||||||
|  |         if (hasAliveEndpoint(snapshot, brokerId, listenerName)) | ||||||
|  |           res.add(brokerId) | ||||||
|  |       } | ||||||
|  |       res | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   // errorUnavailableEndpoints exists to support v0 MetadataResponses | ||||||
|  |   // If errorUnavailableListeners=true, return LISTENER_NOT_FOUND if listener is missing on the broker. | ||||||
|  |   // Otherwise, return LEADER_NOT_AVAILABLE for broker unavailable and missing listener (Metadata response v5 and below). | ||||||
|  |   private def getPartitionMetadata(snapshot: MetadataSnapshot, topic: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean, | ||||||
|  |                                    errorUnavailableListeners: Boolean): Option[Iterable[MetadataResponsePartition]] = { | ||||||
|  |     snapshot.partitionStates.get(topic).map { partitions => | ||||||
|  |       partitions.map { case (partitionId, partitionState) => | ||||||
|  |         val topicPartition = new TopicPartition(topic, partitionId.toInt) | ||||||
|  |         val leaderBrokerId = partitionState.leader | ||||||
|  |         val leaderEpoch = partitionState.leaderEpoch | ||||||
|  |         val maybeLeader = getAliveEndpoint(snapshot, leaderBrokerId, listenerName) | ||||||
|  | 
 | ||||||
|  |         val replicas = partitionState.replicas | ||||||
|  |         val filteredReplicas = maybeFilterAliveReplicas(snapshot, replicas, listenerName, errorUnavailableEndpoints) | ||||||
|  | 
 | ||||||
|  |         val isr = partitionState.isr | ||||||
|  |         val filteredIsr = maybeFilterAliveReplicas(snapshot, isr, listenerName, errorUnavailableEndpoints) | ||||||
|  | 
 | ||||||
|  |         val offlineReplicas = partitionState.offlineReplicas | ||||||
|  | 
 | ||||||
|  |         maybeLeader match { | ||||||
|  |           case None => | ||||||
|  |             val error = if (!snapshot.aliveBrokers.contains(leaderBrokerId)) { // we are already holding the read lock | ||||||
|  |               debug(s"Error while fetching metadata for $topicPartition: leader not available") | ||||||
|  |               Errors.LEADER_NOT_AVAILABLE | ||||||
|  |             } else { | ||||||
|  |               debug(s"Error while fetching metadata for $topicPartition: listener $listenerName " + | ||||||
|  |                 s"not found on leader $leaderBrokerId") | ||||||
|  |               if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             new MetadataResponsePartition() | ||||||
|  |               .setErrorCode(error.code) | ||||||
|  |               .setPartitionIndex(partitionId.toInt) | ||||||
|  |               .setLeaderId(MetadataResponse.NO_LEADER_ID) | ||||||
|  |               .setLeaderEpoch(leaderEpoch) | ||||||
|  |               .setReplicaNodes(filteredReplicas) | ||||||
|  |               .setIsrNodes(filteredIsr) | ||||||
|  |               .setOfflineReplicas(offlineReplicas) | ||||||
|  | 
 | ||||||
|  |           case Some(_) => | ||||||
|  |             val error = if (filteredReplicas.size < replicas.size) { | ||||||
|  |               debug(s"Error while fetching metadata for $topicPartition: replica information not available for " + | ||||||
|  |                 s"following brokers ${replicas.asScala.filterNot(filteredReplicas.contains).mkString(",")}") | ||||||
|  |               Errors.REPLICA_NOT_AVAILABLE | ||||||
|  |             } else if (filteredIsr.size < isr.size) { | ||||||
|  |               debug(s"Error while fetching metadata for $topicPartition: in sync replica information not available for " + | ||||||
|  |                 s"following brokers ${isr.asScala.filterNot(filteredIsr.contains).mkString(",")}") | ||||||
|  |               Errors.REPLICA_NOT_AVAILABLE | ||||||
|  |             } else { | ||||||
|  |               Errors.NONE | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             new MetadataResponsePartition() | ||||||
|  |               .setErrorCode(error.code) | ||||||
|  |               .setPartitionIndex(partitionId.toInt) | ||||||
|  |               .setLeaderId(maybeLeader.map(_.id()).getOrElse(MetadataResponse.NO_LEADER_ID)) | ||||||
|  |               .setLeaderEpoch(leaderEpoch) | ||||||
|  |               .setReplicaNodes(filteredReplicas) | ||||||
|  |               .setIsrNodes(filteredIsr) | ||||||
|  |               .setOfflineReplicas(offlineReplicas) | ||||||
|  |         } | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   /** | ||||||
|  |    * Check whether a broker is alive and has a registered listener matching the provided name. | ||||||
|  |    * This method was added to avoid unnecessary allocations in [[maybeFilterAliveReplicas]], which is | ||||||
|  |    * a hotspot in metadata handling. | ||||||
|  |    */ | ||||||
|  |   private def hasAliveEndpoint(snapshot: MetadataSnapshot, brokerId: Int, listenerName: ListenerName): Boolean = { | ||||||
|  |     snapshot.aliveNodes.get(brokerId).exists(_.contains(listenerName)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   /** | ||||||
|  |    * Get the endpoint matching the provided listener if the broker is alive. Note that listeners can | ||||||
|  |    * be added dynamically, so a broker with a missing listener could be a transient error. | ||||||
|  |    * | ||||||
|  |    * @return None if broker is not alive or if the broker does not have a listener named `listenerName`. | ||||||
|  |    */ | ||||||
|  |   private def getAliveEndpoint(snapshot: MetadataSnapshot, brokerId: Int, listenerName: ListenerName): Option[Node] = { | ||||||
|  |     snapshot.aliveNodes.get(brokerId).flatMap(_.get(listenerName)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   // errorUnavailableEndpoints exists to support v0 MetadataResponses | ||||||
|  |   def getTopicMetadata(topics: Set[String], | ||||||
|  |                        listenerName: ListenerName, | ||||||
|  |                        errorUnavailableEndpoints: Boolean = false, | ||||||
|  |                        errorUnavailableListeners: Boolean = false): Seq[MetadataResponseTopic] = { | ||||||
|  |     val snapshot = metadataSnapshot | ||||||
|  |     topics.toSeq.flatMap { topic => | ||||||
|  |       getPartitionMetadata(snapshot, topic, listenerName, errorUnavailableEndpoints, errorUnavailableListeners).map { partitionMetadata => | ||||||
|  |         new MetadataResponseTopic() | ||||||
|  |           .setErrorCode(Errors.NONE.code) | ||||||
|  |           .setName(topic) | ||||||
|  |           .setTopicId(snapshot.topicIds.getOrElse(topic, Uuid.ZERO_UUID)) | ||||||
|  |           .setIsInternal(Topic.isInternal(topic)) | ||||||
|  |           .setPartitions(partitionMetadata.toBuffer.asJava) | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def getAllTopics(): Set[String] = { | ||||||
|  |     getAllTopics(metadataSnapshot) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def getTopicPartitions(topicName: String): Set[TopicPartition] = { | ||||||
|  |     metadataSnapshot.partitionStates.getOrElse(topicName, Map.empty).values. | ||||||
|  |       map(p => new TopicPartition(topicName, p.partitionIndex())).toSet | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private def getAllTopics(snapshot: MetadataSnapshot): Set[String] = { | ||||||
|  |     snapshot.partitionStates.keySet | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private def getAllPartitions(snapshot: MetadataSnapshot): Map[TopicPartition, UpdateMetadataPartitionState] = { | ||||||
|  |     snapshot.partitionStates.flatMap { case (topic, partitionStates) => | ||||||
|  |       partitionStates.map { case (partition, state ) => (new TopicPartition(topic, partition.toInt), state) } | ||||||
|  |     }.toMap | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def getNonExistingTopics(topics: Set[String]): Set[String] = { | ||||||
|  |     topics.diff(metadataSnapshot.partitionStates.keySet) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def hasAliveBroker(brokerId: Int): Boolean = metadataSnapshot.aliveBrokers.contains(brokerId) | ||||||
|  | 
 | ||||||
|  |   override def getAliveBrokers(): Iterable[BrokerMetadata] = { | ||||||
|  |     metadataSnapshot.aliveBrokers.values.map(b => new BrokerMetadata(b.id, b.rack)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { | ||||||
|  |     metadataSnapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   override def getAliveBrokerNodes(listenerName: ListenerName): Iterable[Node] = { | ||||||
|  |     metadataSnapshot.aliveBrokers.values.flatMap(_.getNode(listenerName)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], | ||||||
|  |                                        topic: String, | ||||||
|  |                                        partitionId: Int, | ||||||
|  |                                        stateInfo: UpdateMetadataPartitionState): Unit = { | ||||||
|  |     val infos = partitionStates.getOrElseUpdate(topic, mutable.LongMap.empty) | ||||||
|  |     infos(partitionId) = stateInfo | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { | ||||||
|  |     metadataSnapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def numPartitions(topic: String): Option[Int] = { | ||||||
|  |     metadataSnapshot.partitionStates.get(topic).map(_.size) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   // if the leader is not known, return None; | ||||||
|  |   // if the leader is known and corresponding node is available, return Some(node) | ||||||
|  |   // if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE) | ||||||
|  |   def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] = { | ||||||
|  |     val snapshot = metadataSnapshot | ||||||
|  |     snapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) map { partitionInfo => | ||||||
|  |       val leaderId = partitionInfo.leader | ||||||
|  | 
 | ||||||
|  |       snapshot.aliveNodes.get(leaderId) match { | ||||||
|  |         case Some(nodeMap) => | ||||||
|  |           nodeMap.getOrElse(listenerName, Node.noNode) | ||||||
|  |         case None => | ||||||
|  |           Node.noNode | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = { | ||||||
|  |     val snapshot = metadataSnapshot | ||||||
|  |     snapshot.partitionStates.get(tp.topic).flatMap(_.get(tp.partition)).map { partitionInfo => | ||||||
|  |       val replicaIds = partitionInfo.replicas | ||||||
|  |       replicaIds.asScala | ||||||
|  |         .map(replicaId => replicaId.intValue() -> { | ||||||
|  |           snapshot.aliveBrokers.get(replicaId.longValue()) match { | ||||||
|  |             case Some(broker) => | ||||||
|  |               broker.getNode(listenerName).getOrElse(Node.noNode()) | ||||||
|  |             case None => | ||||||
|  |               Node.noNode() | ||||||
|  |           }}).toMap | ||||||
|  |         .filter(pair => pair match { | ||||||
|  |           case (_, node) => !node.isEmpty | ||||||
|  |         }) | ||||||
|  |     }.getOrElse(Map.empty[Int, Node]) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def getControllerId: Option[Int] = metadataSnapshot.controllerId | ||||||
|  | 
 | ||||||
|  |   def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster = { | ||||||
|  |     val snapshot = metadataSnapshot | ||||||
|  |     val nodes = snapshot.aliveNodes.flatMap { case (id, nodesByListener) => | ||||||
|  |       nodesByListener.get(listenerName).map { node => | ||||||
|  |         id -> node | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|  |     def node(id: Integer): Node = { | ||||||
|  |       nodes.getOrElse(id.toLong, new Node(id, "", -1)) | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|  |     val partitions = getAllPartitions(snapshot) | ||||||
|  |       .filter { case (_, state) => state.leader != LeaderAndIsr.LeaderDuringDelete } | ||||||
|  |       .map { case (tp, state) => | ||||||
|  |         new PartitionInfo(tp.topic, tp.partition, node(state.leader), | ||||||
|  |           state.replicas.asScala.map(node).toArray, | ||||||
|  |           state.isr.asScala.map(node).toArray, | ||||||
|  |           state.offlineReplicas.asScala.map(node).toArray) | ||||||
|  |       } | ||||||
|  |     val unauthorizedTopics = Collections.emptySet[String] | ||||||
|  |     val internalTopics = getAllTopics(snapshot).filter(Topic.isInternal).asJava | ||||||
|  |     new Cluster(clusterId, nodes.values.toBuffer.asJava, | ||||||
|  |       partitions.toBuffer.asJava, | ||||||
|  |       unauthorizedTopics, internalTopics, | ||||||
|  |       snapshot.controllerId.map(id => node(id)).orNull) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   // This method returns the deleted TopicPartitions received from UpdateMetadataRequest | ||||||
|  |   def updateMetadata(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest): Seq[TopicPartition] = { | ||||||
|  |     inWriteLock(partitionMetadataLock) { | ||||||
|  | 
 | ||||||
|  |       val aliveBrokers = new mutable.LongMap[Broker](metadataSnapshot.aliveBrokers.size) | ||||||
|  |       val aliveNodes = new mutable.LongMap[collection.Map[ListenerName, Node]](metadataSnapshot.aliveNodes.size) | ||||||
|  |       val controllerIdOpt = updateMetadataRequest.controllerId match { | ||||||
|  |         case id if id < 0 => None | ||||||
|  |         case id => Some(id) | ||||||
|  |       } | ||||||
|  | 
 | ||||||
|  |       updateMetadataRequest.liveBrokers.forEach { broker => | ||||||
|  |         // `aliveNodes` is a hot path for metadata requests for large clusters, so we use java.util.HashMap which | ||||||
|  |         // is a bit faster than scala.collection.mutable.HashMap. When we drop support for Scala 2.10, we could | ||||||
|  |         // move to `AnyRefMap`, which has comparable performance. | ||||||
|  |         val nodes = new java.util.HashMap[ListenerName, Node] | ||||||
|  |         val endPoints = new mutable.ArrayBuffer[EndPoint] | ||||||
|  |         broker.endpoints.forEach { ep => | ||||||
|  |           val listenerName = new ListenerName(ep.listener) | ||||||
|  |           endPoints += new EndPoint(ep.host, ep.port, listenerName, SecurityProtocol.forId(ep.securityProtocol)) | ||||||
|  |           nodes.put(listenerName, new Node(broker.id, ep.host, ep.port)) | ||||||
|  |         } | ||||||
|  |         aliveBrokers(broker.id) = Broker(broker.id, endPoints, Option(broker.rack)) | ||||||
|  |         aliveNodes(broker.id) = nodes.asScala | ||||||
|  |       } | ||||||
|  |       aliveNodes.get(brokerId).foreach { listenerMap => | ||||||
|  |         val listeners = listenerMap.keySet | ||||||
|  |         if (!aliveNodes.values.forall(_.keySet == listeners)) | ||||||
|  |           error(s"Listeners are not identical across brokers: $aliveNodes") | ||||||
|  |       } | ||||||
|  | 
 | ||||||
|  |       val newTopicIds = updateMetadataRequest.topicStates().asScala | ||||||
|  |         .map(topicState => (topicState.topicName(), topicState.topicId())) | ||||||
|  |         .filter(_._2 != Uuid.ZERO_UUID).toMap | ||||||
|  |       val topicIds = mutable.Map.empty[String, Uuid] | ||||||
|  |       topicIds ++= metadataSnapshot.topicIds | ||||||
|  |       topicIds ++= newTopicIds | ||||||
|  | 
 | ||||||
|  |       val deletedPartitions = new mutable.ArrayBuffer[TopicPartition] | ||||||
|  |       if (!updateMetadataRequest.partitionStates.iterator.hasNext) { | ||||||
|  |         metadataSnapshot = MetadataSnapshot(metadataSnapshot.partitionStates, topicIds.toMap, controllerIdOpt, aliveBrokers, aliveNodes) | ||||||
|  |       } else { | ||||||
|  |         //since kafka may do partial metadata updates, we start by copying the previous state | ||||||
|  |         val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]](metadataSnapshot.partitionStates.size) | ||||||
|  |         metadataSnapshot.partitionStates.forKeyValue { (topic, oldPartitionStates) => | ||||||
|  |           val copy = new mutable.LongMap[UpdateMetadataPartitionState](oldPartitionStates.size) | ||||||
|  |           copy ++= oldPartitionStates | ||||||
|  |           partitionStates(topic) = copy | ||||||
|  |         } | ||||||
|  | 
 | ||||||
|  |         val traceEnabled = stateChangeLogger.isTraceEnabled | ||||||
|  |         val controllerId = updateMetadataRequest.controllerId | ||||||
|  |         val controllerEpoch = updateMetadataRequest.controllerEpoch | ||||||
|  |         val newStates = updateMetadataRequest.partitionStates.asScala | ||||||
|  |         newStates.foreach { state => | ||||||
|  |           // per-partition logging here can be very expensive due going through all partitions in the cluster | ||||||
|  |           val tp = new TopicPartition(state.topicName, state.partitionIndex) | ||||||
|  |           if (state.leader == LeaderAndIsr.LeaderDuringDelete) { | ||||||
|  |             removePartitionInfo(partitionStates, topicIds, tp.topic, tp.partition) | ||||||
|  |             if (traceEnabled) | ||||||
|  |               stateChangeLogger.trace(s"Deleted partition $tp from metadata cache in response to UpdateMetadata " + | ||||||
|  |                 s"request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") | ||||||
|  |             deletedPartitions += tp | ||||||
|  |           } else { | ||||||
|  |             addOrUpdatePartitionInfo(partitionStates, tp.topic, tp.partition, state) | ||||||
|  |             if (traceEnabled) | ||||||
|  |               stateChangeLogger.trace(s"Cached leader info $state for partition $tp in response to " + | ||||||
|  |                 s"UpdateMetadata request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") | ||||||
|  |           } | ||||||
|  |         } | ||||||
|  |         val cachedPartitionsCount = newStates.size - deletedPartitions.size | ||||||
|  |         stateChangeLogger.info(s"Add $cachedPartitionsCount partitions and deleted ${deletedPartitions.size} partitions from metadata cache " + | ||||||
|  |           s"in response to UpdateMetadata request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") | ||||||
|  | 
 | ||||||
|  |         metadataSnapshot = MetadataSnapshot(partitionStates, topicIds.toMap, controllerIdOpt, aliveBrokers, aliveNodes) | ||||||
|  |       } | ||||||
|  |       deletedPartitions | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def contains(topic: String): Boolean = { | ||||||
|  |     metadataSnapshot.partitionStates.contains(topic) | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   def contains(tp: TopicPartition): Boolean = getPartitionInfo(tp.topic, tp.partition).isDefined | ||||||
|  | 
 | ||||||
|  |   private def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], | ||||||
|  |                                   topicIds: mutable.Map[String, Uuid], topic: String, partitionId: Int): Boolean = { | ||||||
|  |     partitionStates.get(topic).exists { infos => | ||||||
|  |       infos.remove(partitionId) | ||||||
|  |       if (infos.isEmpty) { | ||||||
|  |         partitionStates.remove(topic) | ||||||
|  |         topicIds.remove(topic) | ||||||
|  |       } | ||||||
|  |       true | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  | 
 | ||||||
|  |   case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], | ||||||
|  |                               topicIds: Map[String, Uuid], | ||||||
|  |                               controllerId: Option[Int], | ||||||
|  |                               aliveBrokers: mutable.LongMap[Broker], | ||||||
|  |                               aliveNodes: mutable.LongMap[collection.Map[ListenerName, Node]]) | ||||||
|  | 
 | ||||||
|  | } | ||||||
		Loading…
	
		Reference in New Issue