From aec158e77849e05cc54bc326e5da628e4ea8d082 Mon Sep 17 00:00:00 2001 From: jozi-k Date: Fri, 10 Feb 2017 10:22:00 +0100 Subject: [PATCH 1/3] WIP: KIP-1610: Review usages of Map#mapValues. --- .../main/scala/kafka/admin/AdminClient.scala | 12 +-- .../admin/ReassignPartitionsCommand.scala | 8 +- .../scala/kafka/api/OffsetCommitRequest.scala | 6 +- .../consumer/ConsumerFetcherManager.scala | 67 +++++++------ .../kafka/controller/KafkaController.scala | 1 + .../kafka/coordinator/GroupCoordinator.scala | 4 +- .../kafka/coordinator/GroupMetadata.scala | 16 ++-- .../coordinator/GroupMetadataManager.scala | 18 ++-- .../consumer/ZookeeperConsumerConnector.scala | 1 - .../src/main/scala/kafka/log/LogManager.scala | 20 ++-- .../kafka/server/AbstractFetcherManager.scala | 9 +- .../kafka/server/AbstractFetcherThread.scala | 15 ++- .../scala/kafka/server/DelayedProduce.scala | 2 - .../main/scala/kafka/server/KafkaApis.scala | 53 ++++++---- .../scala/kafka/server/ReplicaManager.scala | 96 ++++++++++--------- .../kafka/tools/ReplicaVerificationTool.scala | 5 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 80 ++++++---------- .../ReplicaFetcherThreadFatalErrorTest.scala | 12 +-- .../GroupCoordinatorResponseTest.scala | 23 ++--- .../GroupMetadataManagerTest.scala | 42 ++++---- .../ZookeeperConsumerConnectorTest.scala | 22 +++-- .../server/AbstractFetcherThreadTest.scala | 17 ++-- .../kafka/server/ReplicaManagerTest.scala | 2 - 23 files changed, 265 insertions(+), 266 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 4b284608a258a..4d9341d217cec 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -121,7 +121,7 @@ class AdminClient(val time: Time, } def listAllGroups(): Map[Node, List[GroupOverview]] = { - findAllBrokers.map { broker => + findAllBrokers().map { broker => broker -> { try { listGroups(broker) @@ -141,11 +141,11 @@ class AdminClient(val time: Time, } def listAllGroupsFlattened(): List[GroupOverview] = { - listAllGroups.values.flatten.toList + listAllGroups().values.flatten.toList } def listAllConsumerGroupsFlattened(): List[GroupOverview] = { - listAllGroupsFlattened.filter(_.protocolType == ConsumerProtocol.PROTOCOL_TYPE) + listAllGroupsFlattened().filter(_.protocolType == ConsumerProtocol.PROTOCOL_TYPE) } def listGroupOffsets(groupId: String): Map[TopicPartition, Long] = { @@ -154,13 +154,13 @@ class AdminClient(val time: Time, val response = responseBody.asInstanceOf[OffsetFetchResponse] if (response.hasError) throw response.error.exception - response.maybeThrowFirstPartitionError + response.maybeThrowFirstPartitionError() response.responseData.asScala.map { case (tp, partitionData) => (tp, partitionData.offset) }.toMap } def listAllBrokerVersionInfo(): Map[Node, Try[NodeApiVersions]] = - findAllBrokers.map { broker => - broker -> Try[NodeApiVersions](new NodeApiVersions(getApiVersions(broker).asJava)) + findAllBrokers().map { broker => + broker -> Try(new NodeApiVersions(getApiVersions(broker).asJava)) }.toMap /** diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 4e7b4e0c3937c..c26beacdbc45b 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -183,7 +183,10 @@ object ReassignPartitionsCommand extends Logging { throw new AdminCommandFailedException("Partition reassignment data file is empty") val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map { case (tp, _) => tp }) if (duplicateReassignedPartitions.nonEmpty) - throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(","))) + throw new AdminCommandFailedException( + s"Partition reassignment contains duplicate topic partitions: ${duplicateReassignedPartitions.mkString(",")}" + ) + val duplicateEntries = partitionsToBeReassigned .map { case (tp, replicas) => (tp, CoreUtils.duplicates(replicas))} .filter { case (_, duplicatedReplicas) => duplicatedReplicas.nonEmpty } @@ -204,7 +207,8 @@ object ReassignPartitionsCommand extends Logging { partitionsToBeReassigned } - private def checkIfReassignmentSucceeded(zkUtils: ZkUtils, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) + private def checkIfReassignmentSucceeded(zkUtils: ZkUtils, + partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) :Map[TopicAndPartition, ReassignmentStatus] = { val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned().mapValues(_.newReplicas) partitionsToBeReassigned.keys.map { topicAndPartition => diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index b9693f67b8e20..348b779140f9e 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -26,8 +26,6 @@ import kafka.network.RequestChannel.Response import kafka.utils.Logging import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import scala.collection._ - object OffsetCommitRequest extends Logging { val CurrentVersion: Short = 2 val DefaultClientId = "" @@ -84,12 +82,12 @@ object OffsetCommitRequest extends Logging { }) }) - OffsetCommitRequest(groupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, memberId, retentionMs) + OffsetCommitRequest(groupId, Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, memberId, retentionMs) } } case class OffsetCommitRequest(groupId: String, - requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata], + requestInfo: Map[TopicAndPartition, OffsetAndMetadata], versionId: Short = OffsetCommitRequest.CurrentVersion, correlationId: Int = 0, clientId: String = OffsetCommitRequest.DefaultClientId, diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index 57a97ef3d554e..c4123750cc2d9 100755 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -21,10 +21,6 @@ import kafka.server.{AbstractFetcherManager, AbstractFetcherThread, BrokerAndIni import kafka.cluster.{BrokerEndPoint, Cluster} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Time - -import scala.collection.immutable -import collection.mutable.HashMap -import scala.collection.mutable import java.util.concurrent.locks.ReentrantLock import kafka.utils.CoreUtils.inLock @@ -43,9 +39,9 @@ class ConsumerFetcherManager(private val consumerIdString: String, private val zkUtils : ZkUtils) extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(Time.SYSTEM.milliseconds), config.clientId, config.numConsumerFetchers) { - private var partitionMap: immutable.Map[TopicPartition, PartitionTopicInfo] = null + private var partitionMap: Map[TopicPartition, PartitionTopicInfo] = null private var cluster: Cluster = null - private val noLeaderPartitionSet = new mutable.HashSet[TopicPartition] + private val noLeaderPartitionSet = new scala.collection.mutable.HashSet[TopicPartition] private val lock = new ReentrantLock private val cond = lock.newCondition() private var leaderFinderThread: ShutdownableThread = null @@ -54,47 +50,56 @@ class ConsumerFetcherManager(private val consumerIdString: String, private class LeaderFinderThread(name: String) extends ShutdownableThread(name) { // thread responsible for adding the fetcher to the right broker when leader is available override def doWork() { - val leaderForPartitionsMap = new HashMap[TopicPartition, BrokerEndPoint] lock.lock() - try { + val leaderForPartitionsMap = try { while (noLeaderPartitionSet.isEmpty) { trace("No partition for leader election.") cond.await() } - trace("Partitions without leader %s".format(noLeaderPartitionSet)) + trace(s"Partitions without leader: $noLeaderPartitionSet") val brokers = ClientUtils.getPlaintextBrokerEndPoints(zkUtils) - val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, - brokers, - config.clientId, - config.socketTimeoutMs, - correlationId.getAndIncrement).topicsMetadata - if(logger.isDebugEnabled) topicsMetadata.foreach(topicMetadata => debug(topicMetadata.toString())) - topicsMetadata.foreach { tmd => - val topic = tmd.topic - tmd.partitionsMetadata.foreach { pmd => - val topicAndPartition = new TopicPartition(topic, pmd.partitionId) - if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) { - val leaderBroker = pmd.leader.get - leaderForPartitionsMap.put(topicAndPartition, leaderBroker) - noLeaderPartitionSet -= topicAndPartition - } - } + val topicsMetadata = + ClientUtils.fetchTopicMetadata( + noLeaderPartitionSet.map(m => m.topic).toSet, + brokers, + config.clientId, + config.socketTimeoutMs, + correlationId.getAndIncrement + ).topicsMetadata + + debug(s"${topicsMetadata.mkString("\n")}") + + val leaderForPartitionSeq = for { + tmd <- topicsMetadata + pmd <- tmd.partitionsMetadata + topicPartition = new TopicPartition(tmd.topic, pmd.partitionId) if noLeaderPartitionSet.contains(topicPartition) + leaderBroker <- pmd.leader + } yield { + noLeaderPartitionSet.remove(topicPartition) + (topicPartition, leaderBroker) } + + leaderForPartitionSeq.toMap } catch { case t: Throwable => { - if (!isRunning.get()) - throw t /* If this thread is stopped, propagate this exception to kill the thread. */ - else - warn("Failed to find leader for %s".format(noLeaderPartitionSet), t) + if (!isRunning.get()) { + throw t /* If this thread is stopped, propagate this exception to kill the thread. */ + } else { + warn(s"Failed to find leader for $noLeaderPartitionSet", t) } + Map.empty[TopicPartition, BrokerEndPoint] + } } finally { lock.unlock() } try { - addFetcherForPartitions(leaderForPartitionsMap.map { case (topicPartition, broker) => - topicPartition -> BrokerAndInitialOffset(broker, partitionMap(topicPartition).getFetchOffset())} + addFetcherForPartitions( + leaderForPartitionsMap.map { + case (topicPartition, broker) => + topicPartition -> BrokerAndInitialOffset(broker, partitionMap(topicPartition).getFetchOffset()) + } ) } catch { case t: Throwable => diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 774316b9f613f..f5659eeac70d6 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1432,6 +1432,7 @@ class PreferredReplicaElectionListener(protected val controller: KafkaController def doHandleDataDeleted(dataPath: String) {} } +//TODO: Very dangerous, if put into map as key and later any var will change, magic will happen!!! case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty, var isrChangeListener: ReassignedPartitionsIsrChangeListener = null) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 891896a705078..f0a7d25d2ea5f 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -19,7 +19,7 @@ package kafka.coordinator import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean -import kafka.common.{OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} +import kafka.common.OffsetAndMetadata import kafka.log.LogConfig import kafka.message.ProducerCompressionCodec import kafka.server._ @@ -29,7 +29,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{OffsetFetchResponse, JoinGroupRequest} -import scala.collection.{Map, Seq, immutable} +import scala.collection.{Seq, immutable} /** diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala index 4ea5bdda76261..75c77c014493c 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala @@ -209,19 +209,15 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou throw new IllegalStateException("Cannot select protocol for empty group") // select the protocol for this group which is supported by all members - val candidates = candidateProtocols - + val candidates = candidateProtocols() // let each member vote for one of the protocols and choose the one with the most votes - val votes: List[(String, Int)] = allMemberMetadata - .map(_.vote(candidates)) - .groupBy(identity) + allMemberMetadata + .groupBy(_.vote(candidates)) .mapValues(_.size) - .toList - - votes.maxBy(_._2)._1 + .maxBy(_._2)._1 } - private def candidateProtocols = { + private def candidateProtocols() = { // get the set of protocols that are commonly supported by all members allMemberMetadata .map(_.protocols) @@ -229,7 +225,7 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou } def supportsProtocols(memberProtocols: Set[String]) = { - members.isEmpty || (memberProtocols & candidateProtocols).nonEmpty + members.isEmpty || (memberProtocols & candidateProtocols()).nonEmpty } def initNextGeneration() = { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index a6ed6a9aad98e..8bbb506baa4c2 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -147,7 +147,9 @@ class GroupMetadataManager(val brokerId: Int, GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion)) val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId)) - val groupMetadataRecords = Map(groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record)) + val groupMetadataRecords = immutable.Map( + groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record) + ) val generationId = group.generationId // set the callback function to insert the created group into cache after log append completed @@ -241,7 +243,9 @@ class GroupMetadataManager(val brokerId: Int, val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId)) - val entries = Map(offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*)) + val entries = immutable.Map( + offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*) + ) // set the callback function to insert offsets into cache after log append completed def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { @@ -326,7 +330,7 @@ class GroupMetadataManager(val brokerId: Int, * The most important guarantee that this API provides is that it should never return a stale offset. i.e., it either * returns the current offset or it begins to sync the cache from the log (and returns an error code). */ - def getOffsets(groupId: String, topicPartitionsOpt: Option[Seq[TopicPartition]]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = { + def getOffsets(groupId: String, topicPartitionsOpt: Option[Seq[TopicPartition]]): immutable.Map[TopicPartition, OffsetFetchResponse.PartitionData] = { trace("Getting offsets of %s for group %s.".format(topicPartitionsOpt.getOrElse("all partitions"), groupId)) val group = groupMetadataCache.get(groupId) if (group == null) { @@ -458,7 +462,7 @@ class GroupMetadataManager(val brokerId: Int, } } - val (groupOffsets, emptyGroupOffsets) = loadedOffsets + val (groupOffsets, emptyGroupOffsets) = loadedOffsets.toMap .groupBy(_._1.group) .mapValues(_.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset)} ) .partition { case (group, _) => loadedGroups.contains(group) } @@ -492,7 +496,7 @@ class GroupMetadataManager(val brokerId: Int, } } - private def loadGroup(group: GroupMetadata, offsets: Map[TopicPartition, OffsetAndMetadata]): Unit = { + private def loadGroup(group: GroupMetadata, offsets: immutable.Map[TopicPartition, OffsetAndMetadata]): Unit = { // offsets are initialized prior to loading the group into the cache to ensure that clients see a consistent // view of the group's offsets val loadedOffsets = offsets.mapValues { offsetAndMetadata => @@ -1072,8 +1076,8 @@ object GroupMetadataManager { } -case class DelayedStore(partitionRecords: Map[TopicPartition, MemoryRecords], - callback: Map[TopicPartition, PartitionResponse] => Unit) +case class DelayedStore(partitionRecords: immutable.Map[TopicPartition, MemoryRecords], + callback: immutable.Map[TopicPartition, PartitionResponse] => Unit) case class GroupTopicPartition(group: String, topicPartition: TopicPartition) { diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index e145075f0b617..2bca64a21503f 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -19,7 +19,6 @@ package kafka.javaapi.consumer import kafka.serializer._ import kafka.consumer._ import kafka.common.{OffsetAndMetadata, TopicAndPartition, MessageStreamsExistException} -import scala.collection.mutable import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 761edf9ac762c..3f14937d8b847 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -340,10 +340,10 @@ class LogManager(val logDirs: Array[File], * Make a checkpoint for all logs in provided directory. */ private def checkpointLogsInDir(dir: File): Unit = { - val recoveryPoints = this.logsByDir.get(dir.toString) - if (recoveryPoints.isDefined) { - this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint)) - } + for { + recoveryPoints <- logsByDir.get(dir.toString) + offsetCheckpoint <- recoveryPointCheckpoints.get(dir) + } yield offsetCheckpoint.write(recoveryPoints.mapValues(_.recoveryPoint)) } /** @@ -447,17 +447,15 @@ class LogManager(val logDirs: Array[File], * data directory with the fewest partitions. */ private def nextLogDir(): File = { - if(logDirs.size == 1) { + if (logDirs.length == 1) { logDirs(0) } else { // count the number of logs in each parent directory (including 0 for empty directories - val logCounts = allLogs.groupBy(_.dir.getParent).mapValues(_.size) + val logCounts = allLogs().groupBy(_.dir.getParent).mapValues(_.size) val zeros = logDirs.map(dir => (dir.getPath, 0)).toMap - var dirCounts = (zeros ++ logCounts).toBuffer - // choose the directory with the least logs in it - val leastLoaded = dirCounts.sortBy(_._2).head - new File(leastLoaded._1) + val leastLoaded = (zeros ++ logCounts).minBy { case (dir, count) => count }._1 + new File(leastLoaded) } } @@ -491,7 +489,7 @@ class LogManager(val logDirs: Array[File], * Map of log dir to logs by topic and partitions in that dir */ private def logsByDir = { - this.logsByTopicPartition.groupBy { + logsByTopicPartition.groupBy { case (_, log) => log.dir.getParent } } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 2b2aa7b900ee1..87d09bf14950b 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -17,9 +17,6 @@ package kafka.server -import scala.collection.mutable -import scala.collection.Set -import scala.collection.Map import kafka.utils.Logging import kafka.cluster.BrokerEndPoint import kafka.metrics.KafkaMetricsGroup @@ -30,7 +27,7 @@ import org.apache.kafka.common.utils.Utils abstract class AbstractFetcherManager(protected val name: String, clientId: String, numFetchers: Int = 1) extends Logging with KafkaMetricsGroup { // map of (source broker_id, fetcher_id per source broker) => fetcher - private val fetcherThreadMap = new mutable.HashMap[BrokerIdAndFetcherId, AbstractFetcherThread] + private val fetcherThreadMap = new scala.collection.mutable.HashMap[BrokerIdAndFetcherId, AbstractFetcherThread] private val mapLock = new Object this.logIdent = "[" + name + "] " @@ -109,12 +106,12 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri for (fetcher <- fetcherThreadMap.values) fetcher.removePartitions(partitions) } - info("Removed fetcher for partitions %s".format(partitions.mkString(","))) + info(s"Removed fetcher for partitions ${partitions.mkString(",")}") } def shutdownIdleFetcherThreads() { mapLock synchronized { - val keysToBeRemoved = new mutable.HashSet[BrokerIdAndFetcherId] + val keysToBeRemoved = new scala.collection.mutable.HashSet[BrokerIdAndFetcherId] for ((key, fetcher) <- fetcherThreadMap) { if (fetcher.partitionCount <= 0) { fetcher.shutdown() diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 0eb3ad83876e6..7aa4cc38a2bf0 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -29,7 +29,6 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.protocol.Errors import AbstractFetcherThread._ -import scala.collection.{Map, Set, mutable} import scala.collection.JavaConverters._ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong @@ -104,7 +103,7 @@ abstract class AbstractFetcherThread(name: String, } private def processFetchRequest(fetchRequest: REQ) { - val partitionsWithError = mutable.Set[TopicPartition]() + val partitionsWithError = scala.collection.mutable.Set[TopicPartition]() def updatePartitionsWithError(partition: TopicPartition): Unit = { partitionsWithError += partition @@ -226,7 +225,7 @@ abstract class AbstractFetcherThread(name: String, for (partition <- partitions) { Option(partitionStates.stateValue(partition)).foreach (currentPartitionFetchState => if (currentPartitionFetchState.isActive) - partitionStates.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) + partitionStates.updateAndMoveToEnd(partition, PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) ) } partitionMapCond.signalAll() @@ -304,11 +303,11 @@ class FetcherLagStats(metricId: ClientIdAndBroker) { val stats = new Pool[ClientIdTopicPartition, FetcherLagMetrics](Some(valueFactory)) def getAndMaybePut(topic: String, partitionId: Int): FetcherLagMetrics = { - stats.getAndMaybePut(new ClientIdTopicPartition(metricId.clientId, topic, partitionId)) + stats.getAndMaybePut(ClientIdTopicPartition(metricId.clientId, topic, partitionId)) } def isReplicaInSync(topic: String, partitionId: Int): Boolean = { - val fetcherLagMetrics = stats.get(new ClientIdTopicPartition(metricId.clientId, topic, partitionId)) + val fetcherLagMetrics = stats.get(ClientIdTopicPartition(metricId.clientId, topic, partitionId)) if (fetcherLagMetrics != null) fetcherLagMetrics.lag <= 0 else @@ -316,7 +315,7 @@ class FetcherLagStats(metricId: ClientIdAndBroker) { } def unregister(topic: String, partitionId: Int) { - val lagMetrics = stats.remove(new ClientIdTopicPartition(metricId.clientId, topic, partitionId)) + val lagMetrics = stats.remove(ClientIdTopicPartition(metricId.clientId, topic, partitionId)) if (lagMetrics != null) lagMetrics.unregister() } @@ -350,9 +349,7 @@ case class ClientIdTopicPartition(clientId: String, topic: String, partitionId: /** * case class to keep partition offset and its state(active, inactive) */ -case class PartitionFetchState(offset: Long, delay: DelayedItem) { - - def this(offset: Long) = this(offset, new DelayedItem(0)) +case class PartitionFetchState(offset: Long, delay: DelayedItem = new DelayedItem(0)) { def isActive: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0 diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index f27dff3142f0b..114a405fa9a47 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -27,8 +27,6 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse -import scala.collection._ - case class ProducePartitionStatus(requiredOffset: Long, responseStatus: PartitionResponse) { @volatile var acksPending = false diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index fa5afe5f68ba4..5b9e4814fa4d3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -246,15 +246,18 @@ class KafkaApis(val requestChannel: RequestChannel, val response = new OffsetCommitResponse(results.asJava) requestChannel.sendResponse(new RequestChannel.Response(request, response)) } else { - val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = offsetCommitRequest.offsetData.asScala.toMap.partition { - case (topicPartition, _) => - val authorizedForDescribe = authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) - val exists = metadataCache.contains(topicPartition.topic) - if (!authorizedForDescribe && exists) + val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = + offsetCommitRequest.offsetData + .asScala + .toMap + .partition { case (topicPartition, _) => + val authorizedForDescribe = authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) + val exists = metadataCache.contains(topicPartition.topic) + if (!authorizedForDescribe && exists) debug(s"Offset commit request with correlation id ${header.correlationId} from client ${header.clientId} " + s"on partition $topicPartition failing due to user not having DESCRIBE authorization, but returning UNKNOWN_TOPIC_OR_PARTITION") - authorizedForDescribe && exists - } + authorizedForDescribe && exists + } val (authorizedTopics, unauthorizedForReadTopics) = existingAndAuthorizedForDescribeTopics.partition { case (topicPartition, _) => authorize(request.session, Read, new Resource(auth.Topic, topicPartition.topic)) @@ -351,9 +354,17 @@ class KafkaApis(val requestChannel: RequestChannel, val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size - val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = produceRequest.partitionRecords.asScala.partition { - case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) && metadataCache.contains(topicPartition.topic) - } + val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = + produceRequest.partitionRecords + .asScala + .toMap + .partition { case (topicPartition, _) => + authorize( + request.session, + Describe, + new Resource(auth.Topic, topicPartition.topic) + ) && metadataCache.contains(topicPartition.topic) + } val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { case (topicPartition, _) => authorize(request.session, Write, new Resource(auth.Topic, topicPartition.topic)) @@ -572,9 +583,13 @@ class KafkaApis(val requestChannel: RequestChannel, val clientId = request.header.clientId val offsetRequest = request.body[ListOffsetRequest] - val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.offsetData.asScala.partition { - case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) - } + val (authorizedRequestInfo, unauthorizedRequestInfo) = + offsetRequest.offsetData + .asScala + .toMap + .partition { case (topicPartition, _) => + authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) + } val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => new ListOffsetResponse.PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION, List[JLong]().asJava) @@ -623,9 +638,13 @@ class KafkaApis(val requestChannel: RequestChannel, val clientId = request.header.clientId val offsetRequest = request.body[ListOffsetRequest] - val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.partitionTimestamps.asScala.partition { - case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) - } + val (authorizedRequestInfo, unauthorizedRequestInfo) = + offsetRequest.partitionTimestamps + .asScala + .toMap + .partition { case (topicPartition, _) => + authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) + } val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => { new ListOffsetResponse.PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION, @@ -1082,7 +1101,7 @@ class KafkaApis(val requestChannel: RequestChannel, syncGroupRequest.groupId(), syncGroupRequest.generationId(), syncGroupRequest.memberId(), - syncGroupRequest.groupAssignment().asScala.mapValues(Utils.toArray), + syncGroupRequest.groupAssignment().asScala.toMap.mapValues(Utils.toArray), sendResponseCallback ) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 1cec4a22a3d1c..69f6d093e59e5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -88,7 +88,7 @@ object LogReadResult { readSize = -1) } -case class BecomeLeaderOrFollowerResult(responseMap: collection.Map[TopicPartition, Errors], error: Errors) { +case class BecomeLeaderOrFollowerResult(responseMap: immutable.Map[TopicPartition, Errors], error: Errors) { override def toString = { "update results: [%s], global error: [%d]".format(responseMap, error.code) @@ -252,12 +252,12 @@ class ReplicaManager(val config: KafkaConfig, .format(localBrokerId, stopReplicaRequest.controllerEpoch, controllerEpoch)) (responseMap, Errors.STALE_CONTROLLER_EPOCH) } else { - val partitions = stopReplicaRequest.partitions.asScala + val partitions = stopReplicaRequest.partitions.asScala.toSet controllerEpoch = stopReplicaRequest.controllerEpoch // First stop fetchers for all partitions, then stop the corresponding replicas replicaFetcherManager.removeFetcherForPartitions(partitions) for (topicPartition <- partitions){ - val error = stopReplica(topicPartition, stopReplicaRequest.deletePartitions) + val error = stopReplica(topicPartition, stopReplicaRequest.deletePartitions()) responseMap.put(topicPartition, error) } (responseMap, Errors.NONE) @@ -301,11 +301,12 @@ class ReplicaManager(val config: KafkaConfig, def appendRecords(timeout: Long, requiredAcks: Short, internalTopicsAllowed: Boolean, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - responseCallback: Map[TopicPartition, PartitionResponse] => Unit) { + entriesPerPartition: immutable.Map[TopicPartition, MemoryRecords], + responseCallback: immutable.Map[TopicPartition, PartitionResponse] => Unit) { if (isValidRequiredAcks(requiredAcks)) { val sTime = time.milliseconds + val localProduceResults = appendToLocalLog(internalTopicsAllowed, entriesPerPartition, requiredAcks) debug("Produce to local log in %d ms".format(time.milliseconds - sTime)) @@ -351,8 +352,8 @@ class ReplicaManager(val config: KafkaConfig, // 2. there is data to append // 3. at least one partition append was successful (fewer errors than partitions) private def delayedRequestRequired(requiredAcks: Short, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = { + entriesPerPartition: immutable.Map[TopicPartition, MemoryRecords], + localProduceResults: immutable.Map[TopicPartition, LogAppendResult]): Boolean = { requiredAcks == -1 && entriesPerPartition.nonEmpty && localProduceResults.values.count(_.exception.isDefined) < entriesPerPartition.size @@ -366,9 +367,9 @@ class ReplicaManager(val config: KafkaConfig, * Append the messages to the local replica logs */ private def appendToLocalLog(internalTopicsAllowed: Boolean, - entriesPerPartition: Map[TopicPartition, MemoryRecords], - requiredAcks: Short): Map[TopicPartition, LogAppendResult] = { - trace("Append [%s] to local log ".format(entriesPerPartition)) + entriesPerPartition: immutable.Map[TopicPartition, MemoryRecords], + requiredAcks: Short): immutable.Map[TopicPartition, LogAppendResult] = { + trace(s"Append [${entriesPerPartition}s] to local log") entriesPerPartition.map { case (topicPartition, records) => BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark() @@ -649,52 +650,57 @@ class ReplicaManager(val config: KafkaConfig, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, topicPartition.topic, topicPartition.partition)) } replicaStateChangeLock synchronized { +// TODO: Try to get rid of mutable map val responseMap = new mutable.HashMap[TopicPartition, Errors] if (leaderAndISRRequest.controllerEpoch < controllerEpoch) { - stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d since " + - "its controller epoch %d is old. Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.controllerId, - correlationId, leaderAndISRRequest.controllerEpoch, controllerEpoch)) - BecomeLeaderOrFollowerResult(responseMap, Errors.STALE_CONTROLLER_EPOCH) + stateChangeLogger.warn( + s"Broker $localBrokerId ignoring LeaderAndIsr request from controller ${leaderAndISRRequest.controllerId} " + + s"with correlation id $correlationId since its controller epoch ${leaderAndISRRequest.controllerEpoch()} is old. " + + s"Latest known controller epoch is $controllerEpoch") + BecomeLeaderOrFollowerResult(Map.empty[TopicPartition, Errors], Errors.STALE_CONTROLLER_EPOCH) } else { val controllerId = leaderAndISRRequest.controllerId controllerEpoch = leaderAndISRRequest.controllerEpoch // First check partition's leader epoch - val partitionState = new mutable.HashMap[Partition, PartitionState]() - leaderAndISRRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) => - val partition = getOrCreatePartition(topicPartition) - val partitionLeaderEpoch = partition.getLeaderEpoch - // If the leader epoch is valid record the epoch of the controller that made the leadership decision. - // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path - if (partitionLeaderEpoch < stateInfo.leaderEpoch) { - if(stateInfo.replicas.contains(localBrokerId)) - partitionState.put(partition, stateInfo) - else { - stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + - "epoch %d for partition [%s,%d] as itself is not in assigned replica list %s") - .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, - topicPartition.topic, topicPartition.partition, stateInfo.replicas.asScala.mkString(","))) - responseMap.put(topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION) - } - } else { - // Otherwise record the error code in response - stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + - "epoch %d for partition [%s,%d] since its associated leader epoch %d is not higher than the current leader epoch %d") - .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, - topicPartition.topic, topicPartition.partition, stateInfo.leaderEpoch, partitionLeaderEpoch)) - responseMap.put(topicPartition, Errors.STALE_CONTROLLER_EPOCH) + val partitionStateMap = leaderAndISRRequest.partitionStates() + .asScala + .foldLeft(Map.empty[Partition, PartitionState]) { + case (partitionAcc, (topicPartition, stateInfo)) => + val partition = getOrCreatePartition(topicPartition) + val partitionLeaderEpoch = partition.getLeaderEpoch + // If the leader epoch is valid record the epoch of the controller that made the leadership decision. + // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path + if (partitionLeaderEpoch < stateInfo.leaderEpoch) { + if(stateInfo.replicas.contains(localBrokerId)) { + partitionAcc + (partition -> stateInfo) + } else { + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + + "epoch %d for partition [%s,%d] as itself is not in assigned replica list %s") + .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, + topicPartition.topic, topicPartition.partition, stateInfo.replicas.asScala.mkString(","))) + responseMap.put(topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION) + partitionAcc + } + } else { + // Otherwise record the error code in response + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + + "epoch %d for partition [%s,%d] since its associated leader epoch %d is not higher than the current leader epoch %d") + .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, + topicPartition.topic, topicPartition.partition, stateInfo.leaderEpoch, partitionLeaderEpoch)) + responseMap.put(topicPartition, Errors.STALE_CONTROLLER_EPOCH) + partitionAcc + } } - } - val partitionsTobeLeader = partitionState.filter { case (_, stateInfo) => - stateInfo.leader == localBrokerId - } - val partitionsToBeFollower = partitionState -- partitionsTobeLeader.keys + val partitionsTobeLeader = partitionStateMap.filter { case (_, stateInfo) => stateInfo.leader == localBrokerId} + val partitionsToBeFollower = partitionStateMap -- partitionsTobeLeader.keys val partitionsBecomeLeader = if (partitionsTobeLeader.nonEmpty) makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, correlationId, responseMap) else Set.empty[Partition] + val partitionsBecomeFollower = if (partitionsToBeFollower.nonEmpty) makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, correlationId, responseMap, metadataCache) else @@ -709,7 +715,7 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.shutdownIdleFetcherThreads() onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower) - BecomeLeaderOrFollowerResult(responseMap, Errors.NONE) + BecomeLeaderOrFollowerResult(responseMap.toMap, Errors.NONE) } } } @@ -729,7 +735,7 @@ class ReplicaManager(val config: KafkaConfig, */ private def makeLeaders(controllerId: Int, epoch: Int, - partitionState: Map[Partition, PartitionState], + partitionState: immutable.Map[Partition, PartitionState], correlationId: Int, responseMap: mutable.Map[TopicPartition, Errors]): Set[Partition] = { partitionState.keys.foreach { partition => @@ -843,7 +849,7 @@ class ReplicaManager(val config: KafkaConfig, } } - replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(_.topicPartition)) + replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(_.topicPartition).toSet) partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-follower request from controller " + "%d epoch %d with correlation id %d for partition %s") diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 492b3042114cf..611f78e183b00 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -34,9 +34,6 @@ import kafka.utils._ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.utils.Time -import scala.collection.JavaConverters._ - - /** * For verifying the consistency among replicas. * @@ -106,7 +103,7 @@ object ReplicaVerificationTool extends Logging { CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt) val regex = options.valueOf(topicWhiteListOpt) - val topicWhiteListFiler = new Whitelist(regex) + val topicWhiteListFiler = Whitelist(regex) try { Pattern.compile(regex) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 7a6bd6336b886..f17d0bc5ce8b3 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -31,9 +31,6 @@ import org.I0Itec.zkclient.exception.{ZkBadVersionException, ZkException, ZkMars import org.I0Itec.zkclient.serialize.ZkSerializer import org.I0Itec.zkclient.{ZkClient, ZkConnection} import org.apache.kafka.common.config.ConfigException -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.SecurityProtocol -import org.apache.kafka.common.utils.Time import org.apache.zookeeper.AsyncCallback.{DataCallback, StringCallback} import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.data.{ACL, Stat} @@ -128,45 +125,32 @@ object ZkUtils { def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic - // Parses without deduplicating keys so the data can be checked before allowing reassignment to proceed + // TODO: Does this comment mean we cannot get duplicates of TopicAndPartition in the resulting Seq? + /** Parses without deduplicate keys so the data can be checked before allowing reassignment to proceed */ def parsePartitionReassignmentDataWithoutDedup(jsonData: String): Seq[(TopicAndPartition, Seq[Int])] = { - Json.parseFull(jsonData) match { - case Some(m) => - m.asInstanceOf[Map[String, Any]].get("partitions") match { - case Some(partitionsSeq) => - partitionsSeq.asInstanceOf[Seq[Map[String, Any]]].map(p => { - val topic = p.get("topic").get.asInstanceOf[String] - val partition = p.get("partition").get.asInstanceOf[Int] - val newReplicas = p.get("replicas").get.asInstanceOf[Seq[Int]] - TopicAndPartition(topic, partition) -> newReplicas - }) - case None => - Seq.empty - } - case None => - Seq.empty + for { + json <- Json.parseFull(jsonData).toSeq + partitionsSeq <- json.asInstanceOf[Map[String, Any]].get("partitions").toSeq + p <- partitionsSeq.asInstanceOf[Seq[Map[String, Any]]] + } yield { +// TODO: Option#get throws exception for None, we should do something meaningful. + val topic = p.get("topic").get.asInstanceOf[String] + val partition = p.get("partition").get.asInstanceOf[Int] + val newReplicas = p.get("replicas").get.asInstanceOf[Seq[Int]] + TopicAndPartition(topic, partition) -> newReplicas } } - def parsePartitionReassignmentData(jsonData: String): Map[TopicAndPartition, Seq[Int]] = + def parsePartitionReassignmentData(jsonData: String): immutable.Map[TopicAndPartition, Seq[Int]] = parsePartitionReassignmentDataWithoutDedup(jsonData).toMap def parseTopicsData(jsonData: String): Seq[String] = { - var topics = List.empty[String] - Json.parseFull(jsonData) match { - case Some(m) => - m.asInstanceOf[Map[String, Any]].get("topics") match { - case Some(partitionsSeq) => - val mapPartitionSeq = partitionsSeq.asInstanceOf[Seq[Map[String, Any]]] - mapPartitionSeq.foreach(p => { - val topic = p.get("topic").get.asInstanceOf[String] - topics ++= List(topic) - }) - case None => - } - case None => - } - topics + for { + m <- Json.parseFull(jsonData).toSeq + partitionsSeq <- m.asInstanceOf[Map[String, Any]].get("topics").toSeq + p <- partitionsSeq.asInstanceOf[Seq[Map[String, Any]]] +// TODO: What if "topic" is not part of json? None#get throws exception. + } yield p.get("topic").get.asInstanceOf[String] } def formatAsReassignmentJson(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = { @@ -699,23 +683,21 @@ class ZkUtils(val zkClient: ZkClient, } def getPartitionsForTopics(topics: Seq[String]): mutable.Map[String, Seq[Int]] = { - getPartitionAssignmentForTopics(topics).map { topicAndPartitionMap => - val topic = topicAndPartitionMap._1 - val partitionMap = topicAndPartitionMap._2 - debug("partition assignment of /brokers/topics/%s is %s".format(topic, partitionMap)) - topic -> partitionMap.keys.toSeq.sortWith((s, t) => s < t) + getPartitionAssignmentForTopics(topics).map { + case (topic, partitionMap) => + debug(s"partition assignment of /brokers/topics/$topic is $partitionMap") + topic -> partitionMap.keys.toSeq.sortWith((s, t) => s < t) } } - def getPartitionsBeingReassigned(): Map[TopicAndPartition, ReassignedPartitionsContext] = { - // read the partitions and their new replica list - val jsonPartitionMapOpt = readDataMaybeNull(ReassignPartitionsPath)._1 - jsonPartitionMapOpt match { - case Some(jsonPartitionMap) => - val reassignedPartitions = parsePartitionReassignmentData(jsonPartitionMap) - reassignedPartitions.map(p => p._1 -> new ReassignedPartitionsContext(p._2)) - case None => Map.empty[TopicAndPartition, ReassignedPartitionsContext] - } + /** Read the partitions and their new replica list */ + def getPartitionsBeingReassigned(): immutable.Map[TopicAndPartition, ReassignedPartitionsContext] = { + val reassignedPartitions = for { + jsonPartitionMap <- readDataMaybeNull(ReassignPartitionsPath)._1.toSeq + (tp, replicas) <- parsePartitionReassignmentData(jsonPartitionMap) + } yield tp -> ReassignedPartitionsContext(replicas) + + reassignedPartitions.toMap } def updatePartitionReassignmentData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) { diff --git a/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala b/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala index 853dad608acde..3ce16b5d7e1f3 100644 --- a/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala +++ b/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala @@ -33,13 +33,12 @@ import org.apache.kafka.common.requests.FetchResponse import org.apache.kafka.common.utils.Time import org.junit.{After, Test} -import scala.collection.Map import scala.collection.JavaConverters._ import scala.concurrent.Future class ReplicaFetcherThreadFatalErrorTest extends ZooKeeperTestHarness { - private var brokers: Seq[KafkaServer] = null + private var brokers = Seq.empty[KafkaServer] @volatile private var shutdownCompleted = false @After @@ -96,7 +95,7 @@ class ReplicaFetcherThreadFatalErrorTest extends ZooKeeperTestHarness { } } })) - TestUtils.createTopic(zkUtils, "topic", numPartitions = 1, replicationFactor = 2, servers = brokers) + TestUtils.createTopic(zkUtils, "topic", replicationFactor = 2, servers = brokers) TestUtils.waitUntilTrue(() => shutdownCompleted, "Shutdown of follower did not complete") } @@ -116,10 +115,11 @@ class ReplicaFetcherThreadFatalErrorTest extends ZooKeeperTestHarness { quotaManager: ReplicationQuotaManager) = new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager) { override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { - val prefix = threadNamePrefix.map(tp => s"${tp}:").getOrElse("") + val prefix = threadNamePrefix.map(tp => s"$tp:").getOrElse("") val threadName = s"${prefix}ReplicaFetcherThread-$fetcherId-${sourceBroker.id}" - fetcherThread(new FetcherThreadParams(threadName, fetcherId, sourceBroker, replicaManager, metrics, - time, quotaManager)) + fetcherThread( + FetcherThreadParams(threadName, fetcherId, sourceBroker, replicaManager, metrics, time, quotaManager) + ) } } } diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 7b401875e9855..7e3608009b8d1 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -32,7 +32,6 @@ import org.junit.{After, Before, Test} import org.scalatest.junit.JUnitSuite import java.util.concurrent.TimeUnit -import scala.collection._ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future, Promise} @@ -79,7 +78,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) // make two partitions of the group topic to make sure some partitions are not owned by the coordinator - val ret = mutable.Map[String, Map[Int, Seq[Int]]]() + val ret = scala.collection.mutable.Map[String, scala.collection.Map[Int, Seq[Int]]]() ret += (Topic.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) @@ -366,7 +365,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { timer.advanceClock(sessionTimeout / 2) EasyMock.reset(replicaManager) - val commitOffsetResult = commitOffsets(groupId, assignedConsumerId, generationId, immutable.Map(tp -> offset)) + val commitOffsetResult = commitOffsets(groupId, assignedConsumerId, generationId, Map(tp -> offset)) assertEquals(Errors.NONE, commitOffsetResult(tp)) timer.advanceClock(sessionTimeout / 2 + 100) @@ -734,7 +733,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val tp = new TopicPartition("topic", 0) val offset = OffsetAndMetadata(0) - val commitOffsetResult = commitOffsets(groupId, memberId, generationId, immutable.Map(tp -> offset)) + val commitOffsetResult = commitOffsets(groupId, memberId, generationId, Map(tp -> offset)) assertEquals(Errors.ILLEGAL_GENERATION, commitOffsetResult(tp)) } @@ -744,7 +743,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val offset = OffsetAndMetadata(0) val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, - OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset)) + OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset)) assertEquals(Errors.NONE, commitOffsetResult(tp)) } @@ -754,7 +753,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val offset = OffsetAndMetadata(0) val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, - OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset)) + OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset)) assertEquals(Errors.NONE, commitOffsetResult(tp)) val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp))) @@ -790,7 +789,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals((Errors.NONE, Map.empty), groupCoordinator.handleFetchOffsets(groupId)) val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, - OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp1 -> offset1, tp2 -> offset2, tp3 -> offset3)) + OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp1 -> offset1, tp2 -> offset2, tp3 -> offset3)) assertEquals(Errors.NONE, commitOffsetResult(tp1)) assertEquals(Errors.NONE, commitOffsetResult(tp2)) assertEquals(Errors.NONE, commitOffsetResult(tp3)) @@ -817,7 +816,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE, joinGroupError) EasyMock.reset(replicaManager) - val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, immutable.Map(tp -> offset)) + val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, Map(tp -> offset)) assertEquals(Errors.REBALANCE_IN_PROGRESS, commitOffsetResult(tp)) } @@ -1049,7 +1048,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.anyShort(), EasyMock.anyBoolean(), EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]], - EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { + EasyMock.capture(capturedArgument)) + ).andAnswer(new IAnswer[Unit] { override def answer = capturedArgument.getValue.apply( Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> new PartitionResponse(Errors.NONE, 0L, Record.NO_TIMESTAMP) @@ -1122,7 +1122,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def commitOffsets(groupId: String, consumerId: String, generationId: Int, - offsets: immutable.Map[TopicPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = { + offsets: Map[TopicPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture() @@ -1131,7 +1131,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.anyShort(), EasyMock.anyBoolean(), EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]], - EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { + EasyMock.capture(capturedArgument)) + ).andAnswer(new IAnswer[Unit] { override def answer = capturedArgument.getValue.apply( Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> new PartitionResponse(Errors.NONE, 0L, Record.NO_TIMESTAMP) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala index 86189aa0cf186..64b51c5b12cbb 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala @@ -35,8 +35,7 @@ import org.junit.{Before, Test} import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import kafka.utils.TestUtils.fail -import scala.collection._ -import JavaConverters._ +import scala.collection.JavaConverters._ class GroupMetadataManagerTest { @@ -69,7 +68,8 @@ class GroupMetadataManagerTest { offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) // make two partitions of the group topic to make sure some partitions are not owned by the coordinator - val ret = mutable.Map[String, Map[Int, Seq[Int]]]() +// TODO: Get rid of mutable and collection + val ret = scala.collection.mutable.Map[String, scala.collection.Map[Int, Seq[Int]]]() ret += (Topic.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) @@ -359,13 +359,13 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId) groupMetadataManager.addGroup(group) - val offsets = immutable.Map(topicPartition -> OffsetAndMetadata(offset)) + val offsets = Map(topicPartition -> OffsetAndMetadata(offset)) expectAppendMessage(Errors.NONE) EasyMock.replay(replicaManager) - var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None - def callback(errors: immutable.Map[TopicPartition, Errors]) { + var commitErrors: Option[Map[TopicPartition, Errors]] = None + def callback(errors: Map[TopicPartition, Errors]) { commitErrors = Some(errors) } @@ -401,12 +401,12 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId) groupMetadataManager.addGroup(group) - val offsets = immutable.Map(topicPartition -> OffsetAndMetadata(offset)) + val offsets = Map(topicPartition -> OffsetAndMetadata(offset)) EasyMock.replay(replicaManager) - var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None - def callback(errors: immutable.Map[TopicPartition, Errors]) { + var commitErrors: Option[Map[TopicPartition, Errors]] = None + def callback(errors: Map[TopicPartition, Errors]) { commitErrors = Some(errors) } @@ -443,13 +443,13 @@ class GroupMetadataManagerTest { val group = new GroupMetadata(groupId) groupMetadataManager.addGroup(group) - val offsets = immutable.Map(topicPartition -> OffsetAndMetadata(offset)) + val offsets = Map(topicPartition -> OffsetAndMetadata(offset)) expectAppendMessage(appendError) EasyMock.replay(replicaManager) - var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None - def callback(errors: immutable.Map[TopicPartition, Errors]) { + var commitErrors: Option[Map[TopicPartition, Errors]] = None + def callback(errors: Map[TopicPartition, Errors]) { commitErrors = Some(errors) } @@ -484,7 +484,7 @@ class GroupMetadataManagerTest { // expire the offset after 1 millisecond val startMs = time.milliseconds - val offsets = immutable.Map( + val offsets = Map( topicPartition1 -> OffsetAndMetadata(offset, "", startMs, startMs + 1), topicPartition2 -> OffsetAndMetadata(offset, "", startMs, startMs + 3)) @@ -492,8 +492,8 @@ class GroupMetadataManagerTest { expectAppendMessage(Errors.NONE) EasyMock.replay(replicaManager) - var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None - def callback(errors: immutable.Map[TopicPartition, Errors]) { + var commitErrors: Option[Map[TopicPartition, Errors]] = None + def callback(errors: Map[TopicPartition, Errors]) { commitErrors = Some(errors) } @@ -633,7 +633,7 @@ class GroupMetadataManagerTest { // expire the offset after 1 millisecond val startMs = time.milliseconds - val offsets = immutable.Map( + val offsets = Map( topicPartition1 -> OffsetAndMetadata(offset, "", startMs, startMs + 1), topicPartition2 -> OffsetAndMetadata(offset, "", startMs, startMs + 3)) @@ -641,8 +641,8 @@ class GroupMetadataManagerTest { expectAppendMessage(Errors.NONE) EasyMock.replay(replicaManager) - var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None - def callback(errors: immutable.Map[TopicPartition, Errors]) { + var commitErrors: Option[Map[TopicPartition, Errors]] = None + def callback(errors: Map[TopicPartition, Errors]) { commitErrors = Some(errors) } @@ -709,7 +709,7 @@ class GroupMetadataManagerTest { // expire the offset after 1 millisecond val startMs = time.milliseconds - val offsets = immutable.Map( + val offsets = Map( topicPartition1 -> OffsetAndMetadata(offset, "", startMs, startMs + 1), topicPartition2 -> OffsetAndMetadata(offset, "", startMs, startMs + 3)) @@ -717,8 +717,8 @@ class GroupMetadataManagerTest { expectAppendMessage(Errors.NONE) EasyMock.replay(replicaManager) - var commitErrors: Option[immutable.Map[TopicPartition, Errors]] = None - def callback(errors: immutable.Map[TopicPartition, Errors]) { + var commitErrors: Option[Map[TopicPartition, Errors]] = None + def callback(errors: Map[TopicPartition, Errors]) { commitErrors = Some(errors) } diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index 7d8e0c2e02b7e..3f260849740d5 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -27,11 +27,10 @@ import kafka.javaapi.producer.Producer import kafka.utils.IntEncoder import kafka.utils.{Logging, TestUtils} import kafka.consumer.{KafkaStream, ConsumerConfig} -import kafka.zk.ZooKeeperTestHarness import kafka.common.MessageStreamsExistException import org.junit.Test -import scala.collection.JavaConversions +import scala.collection.JavaConverters._ import org.apache.log4j.{Level, Logger} import org.junit.Assert._ @@ -65,7 +64,14 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(toJavaMap(Map(topic -> numNodes*numParts/2)), new StringDecoder(), new StringDecoder()) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams( + toJavaMap(Map(topic -> numNodes*numParts/2)), + new StringDecoder(), + new StringDecoder() + ).asScala + .toMap + .mapValues(_.asScala.toList) + val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1) assertEquals(sentMessages1.sorted, receivedMessages1.sorted) @@ -95,8 +101,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging for (partition <- 0 until numParts) { val ms = 0.until(messagesPerNode).map(x => header + server.config.brokerId + "-" + partition + "-" + x) messages ++= ms - import JavaConversions._ - javaProducer.send(ms.map(new KeyedMessage[Int, String](topic, partition, _)): java.util.List[KeyedMessage[Int, String]]) + javaProducer.send(ms.map(new KeyedMessage[Int, String](topic, partition, _)).asJava) } javaProducer.close } @@ -104,12 +109,9 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging } def getMessages(nMessagesPerThread: Int, - jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[String, String]]]): List[String] = { - var messages: List[String] = Nil - import scala.collection.JavaConversions._ + jTopicMessageStreams: Map[String, List[KafkaStream[String, String]]]): List[String] = { val topicMessageStreams = jTopicMessageStreams.mapValues(_.toList) - messages = TestUtils.getMessages(topicMessageStreams, nMessagesPerThread) - messages + TestUtils.getMessages(topicMessageStreams, nMessagesPerThread) } private def toJavaMap(scalaMap: Map[String, Int]): java.util.Map[String, java.lang.Integer] = { diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index f279c856408f0..b3d8e36dc4f5e 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -29,7 +29,6 @@ import org.junit.Assert.{assertFalse, assertTrue} import org.junit.{Before, Test} import scala.collection.JavaConverters._ -import scala.collection.{Map, mutable} class AbstractFetcherThreadTest { @@ -85,7 +84,7 @@ class AbstractFetcherThreadTest { private def allMetricsNames = Metrics.defaultRegistry().allMetrics().asScala.keySet.map(_.getName) - class DummyFetchRequest(val offsets: collection.Map[TopicPartition, Long]) extends FetchRequest { + class DummyFetchRequest(val offsets: Map[TopicPartition, Long]) extends FetchRequest { override def isEmpty: Boolean = offsets.isEmpty override def offset(topicPartition: TopicPartition): Long = offsets(topicPartition) @@ -121,7 +120,7 @@ class AbstractFetcherThreadTest { override protected def fetch(fetchRequest: DummyFetchRequest): Seq[(TopicPartition, TestPartitionData)] = fetchRequest.offsets.mapValues(_ => new TestPartitionData()).toSeq - override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = + override protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap) } @@ -192,14 +191,12 @@ class AbstractFetcherThreadTest { fetchRequest.offsets.mapValues(v => normalPartitionDataSet(v.toInt)).toSeq } - override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = { - val requestMap = new mutable.HashMap[TopicPartition, Long] - partitionMap.foreach { case (topicPartition, partitionFetchState) => - // Add backoff delay check - if (partitionFetchState.isActive) - requestMap.put(topicPartition, partitionFetchState.offset) + override protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = { + val topicPartitionOffsets = partitionMap.collect { + case (topicPartition, partitionFetchState) if partitionFetchState.isActive => + topicPartition -> partitionFetchState.offset } - new DummyFetchRequest(requestMap) + new DummyFetchRequest(topicPartitionOffsets.toMap) } override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) = delayPartitions(partitions, fetchBackOffMs.toLong) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index c481ac41a3a7b..99d245f8c2277 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -21,7 +21,6 @@ import java.io.File import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean -import kafka.cluster.Broker import kafka.log.LogConfig import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils} import TestUtils.createBroker @@ -38,7 +37,6 @@ import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ -import scala.collection.Map class ReplicaManagerTest { From c121bcc16b66e176076de265e126b804acecc5f1 Mon Sep 17 00:00:00 2001 From: jozi-k Date: Thu, 16 Feb 2017 17:32:16 +0100 Subject: [PATCH 2/3] Change to immutable collections and simplify LeaderAndIsr. --- .../main/scala/kafka/api/LeaderAndIsr.scala | 34 +++- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../controller/ControllerChannelManager.scala | 22 ++- .../kafka/controller/KafkaController.scala | 171 ++++++++++-------- .../controller/PartitionLeaderSelector.scala | 46 ++--- .../controller/PartitionStateMachine.scala | 16 +- .../controller/ReplicaStateMachine.scala | 15 +- .../controller/TopicDeletionManager.scala | 26 +-- .../scala/kafka/server/MetadataCache.scala | 2 +- ...ControlledShutdownLeaderSelectorTest.scala | 2 +- .../kafka/server/LeaderElectionTest.scala | 3 +- .../scala/unit/kafka/utils/TestUtils.scala | 2 +- 12 files changed, 195 insertions(+), 146 deletions(-) diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala index e68ad866834ca..6930e7316b490 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala @@ -19,26 +19,48 @@ package kafka.api import kafka.controller.LeaderIsrAndControllerEpoch import kafka.utils._ +import org.apache.kafka.common.requests.PartitionState import scala.collection.Set +import scala.collection.JavaConverters._ object LeaderAndIsr { val initialLeaderEpoch: Int = 0 val initialZKVersion: Int = 0 - val NoLeader = -1 - val LeaderDuringDelete = -2 + val NoLeader: Int = -1 + val LeaderDuringDelete: Int = -2 + + def apply(leader: Int, isr: List[Int]): LeaderAndIsr = LeaderAndIsr(leader, initialLeaderEpoch, isr, initialZKVersion) + + def apply(partitionState: PartitionState): LeaderAndIsr = { + LeaderAndIsr( + partitionState.leader, + partitionState.leaderEpoch, + partitionState.isr.asScala.map(_.toInt).toList, + partitionState.zkVersion + ) + } } -case class LeaderAndIsr(var leader: Int, var leaderEpoch: Int, var isr: List[Int], var zkVersion: Int) { - def this(leader: Int, isr: List[Int]) = this(leader, LeaderAndIsr.initialLeaderEpoch, isr, LeaderAndIsr.initialZKVersion) +//TODO: Do not use vars!!!, change isr to Set[Int] +case class LeaderAndIsr(var leader: Int, + var leaderEpoch: Int, + var isr: List[Int], + var zkVersion: Int) { + def newLeader(newLeader: Int) = LeaderAndIsr(newLeader, leaderEpoch + 1, isr, zkVersion + 1) + + def newLeaderAndIsr(newLeader: Int, newIsr: List[Int]) = + LeaderAndIsr(newLeader, leaderEpoch + 1, newIsr, zkVersion + 1) + + def newEpochAndZkVersion = LeaderAndIsr(leader, leaderEpoch + 1, isr, zkVersion + 1) override def toString: String = { Json.encode(Map("leader" -> leader, "leader_epoch" -> leaderEpoch, "isr" -> isr)) } } -case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Set[Int]) { - +case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, + allReplicas: Set[Int]) { def replicationFactor = allReplicas.size override def toString: String = { diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index c2d34d9552f4a..9e51faea85003 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -468,7 +468,7 @@ class Partition(val topic: String, } private def updateIsr(newIsr: Set[Replica]) { - val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) + val newLeaderAndIsr = LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partitionId, newLeaderAndIsr, controllerEpoch, zkVersion) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index a2fee6b6b4e1c..7ae85db6a5ecf 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -36,11 +36,14 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, TopicPartition} import scala.collection.JavaConverters._ -import scala.collection.{Set, mutable} -import scala.collection.mutable.HashMap - -class ControllerChannelManager(controllerContext: ControllerContext, config: KafkaConfig, time: Time, metrics: Metrics, threadNamePrefix: Option[String] = None) extends Logging { - protected val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] +import scala.collection.mutable + +class ControllerChannelManager(controllerContext: ControllerContext, + config: KafkaConfig, + time: Time, + metrics: Metrics, + threadNamePrefix: Option[String] = None) extends Logging { + protected val brokerStateInfo = new scala.collection.mutable.HashMap[Int, ControllerBrokerStateInfo] private val brokerLock = new Object this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " @@ -132,7 +135,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf val requestThread = new RequestSendThread(config.brokerId, controllerContext, messageQueue, networkClient, brokerNode, config, time, threadName) requestThread.setDaemon(false) - brokerStateInfo.put(broker.id, new ControllerBrokerStateInfo(networkClient, brokerNode, messageQueue, requestThread)) + brokerStateInfo.put(broker.id, ControllerBrokerStateInfo(networkClient, brokerNode, messageQueue, requestThread)) } private def removeExistingBroker(brokerState: ControllerBrokerStateInfo) { @@ -248,7 +251,7 @@ class RequestSendThread(val controllerId: Int, } -class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging { +class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging { val controllerContext = controller.controllerContext val controllerId: Int = controller.config.brokerId val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, PartitionStateInfo]] @@ -308,7 +311,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging /** Send UpdateMetadataRequest to the given brokers for the given partitions and partitions that are being deleted */ def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], - partitions: collection.Set[TopicAndPartition] = Set.empty[TopicAndPartition], + partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition], callback: AbstractResponse => Unit = null) { def updateMetadataRequestPartitionInfo(partition: TopicAndPartition, beingDeleted: Boolean) { val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) @@ -316,7 +319,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging case Some(leaderIsrAndControllerEpoch) => val replicas = controllerContext.partitionReplicaAssignment(partition).toSet val partitionStateInfo = if (beingDeleted) { - val leaderAndIsr = new LeaderAndIsr(LeaderAndIsr.LeaderDuringDelete, leaderIsrAndControllerEpoch.leaderAndIsr.isr) + val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.LeaderDuringDelete, leaderIsrAndControllerEpoch.leaderAndIsr.isr) PartitionStateInfo(LeaderIsrAndControllerEpoch(leaderAndIsr, leaderIsrAndControllerEpoch.controllerEpoch), replicas) } else { PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) @@ -332,6 +335,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging controllerContext.partitionLeadershipInfo.keySet else partitions + if (controller.deleteTopicManager.partitionsToBeDeleted.isEmpty) givenPartitions else diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index f5659eeac70d6..98e23af524705 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -79,11 +79,11 @@ class ControllerContext(val zkUtils: ZkUtils) { }.toSet } - def replicasOnBrokers(brokerIds: Set[Int]): Set[PartitionAndReplica] = { + def replicasOnBrokers(brokerIds: immutable.Set[Int]): immutable.Set[PartitionAndReplica] = { brokerIds.flatMap { brokerId => partitionReplicaAssignment.collect { case (topicAndPartition, replicas) if replicas.contains(brokerId) => - new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, brokerId) + PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, brokerId) } }.toSet } @@ -93,7 +93,7 @@ class ControllerContext(val zkUtils: ZkUtils) { .filter { case (topicAndPartition, _) => topicAndPartition.topic == topic } .flatMap { case (topicAndPartition, replicas) => replicas.map { r => - new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, r) + PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, r) } }.toSet } @@ -101,14 +101,14 @@ class ControllerContext(val zkUtils: ZkUtils) { def partitionsForTopic(topic: String): collection.Set[TopicAndPartition] = partitionReplicaAssignment.keySet.filter(topicAndPartition => topicAndPartition.topic == topic) - def allLiveReplicas(): Set[PartitionAndReplica] = { - replicasOnBrokers(liveBrokerIds) + def allLiveReplicas(): immutable.Set[PartitionAndReplica] = { + replicasOnBrokers(liveBrokerIds.toSet) } - def replicasForPartition(partitions: collection.Set[TopicAndPartition]): collection.Set[PartitionAndReplica] = { + def replicasForPartition(partitions: immutable.Set[TopicAndPartition]): immutable.Set[PartitionAndReplica] = { partitions.flatMap { p => val replicas = partitionReplicaAssignment(p) - replicas.map(r => new PartitionAndReplica(p.topic, p.partition, r)) + replicas.map(r => PartitionAndReplica(p.topic, p.partition, r)) } } @@ -286,8 +286,13 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState } } // If the broker is a follower, updates the isr in ZK and notifies the current leader - replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, - topicAndPartition.partition, id)), OfflineReplica) + replicaStateMachine.handleStateChanges( + immutable.Set( + PartitionAndReplica(topicAndPartition.topic, + topicAndPartition.partition, id) + ), + OfflineReplica + ) } } } @@ -506,8 +511,8 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState * 2. Invokes the new partition callback * 3. Send metadata request with the new topic to all brokers so they allow requests for that topic to be served */ - def onNewTopicCreation(topics: Set[String], newPartitions: Set[TopicAndPartition]) { - info("New topic creation callback for %s".format(newPartitions.mkString(","))) + def onNewTopicCreation(topics: Set[String], newPartitions: immutable.Set[TopicAndPartition]) { + info(s"New topic creation callback for ${newPartitions.mkString(",")}") // subscribe to partition changes topics.foreach(topic => partitionStateMachine.registerPartitionChangeListener(topic)) onNewPartitionCreation(newPartitions) @@ -519,7 +524,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState * 1. Move the newly created partitions to the NewPartition state * 2. Move the newly created partitions from NewPartition->OnlinePartition state */ - def onNewPartitionCreation(newPartitions: Set[TopicAndPartition]) { + def onNewPartitionCreation(newPartitions: immutable.Set[TopicAndPartition]) { info("New partition creation callback for %s".format(newPartitions.mkString(","))) partitionStateMachine.handleStateChanges(newPartitions, NewPartition) replicaStateMachine.handleStateChanges(controllerContext.replicasForPartition(newPartitions), NewReplica) @@ -568,7 +573,8 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState * Note that we have to update AR in ZK with RAR last since it's the only place where we store OAR persistently. * This way, if the controller crashes before that step, we can still recover. */ - def onPartitionReassignment(topicAndPartition: TopicAndPartition, reassignedPartitionContext: ReassignedPartitionsContext) { + def onPartitionReassignment(topicAndPartition: TopicAndPartition, + reassignedPartitionContext: ReassignedPartitionsContext) { val reassignedReplicas = reassignedPartitionContext.newReplicas if (!areReplicasInIsr(topicAndPartition.topic, topicAndPartition.partition, reassignedReplicas)) { info("New replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + @@ -581,24 +587,29 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState updateLeaderEpochAndSendRequest(topicAndPartition, controllerContext.partitionReplicaAssignment(topicAndPartition), newAndOldReplicas.toSeq) //3. replicas in RAR - OAR -> NewReplica - startNewReplicasForReassignedPartition(topicAndPartition, reassignedPartitionContext, newReplicasNotInOldReplicaList) + startNewReplicasForReassignedPartition(topicAndPartition, newReplicasNotInOldReplicaList) info("Waiting for new replicas %s for partition %s being ".format(reassignedReplicas.mkString(","), topicAndPartition) + "reassigned to catch up with the leader") } else { //4. Wait until all replicas in RAR are in sync with the leader. val oldReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).toSet -- reassignedReplicas.toSet //5. replicas in RAR -> OnlineReplica +// TODO: This lambda is repeating in startNewReplicasForReassignedPartition reassignedReplicas.foreach { replica => - replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, - replica)), OnlineReplica) + replicaStateMachine.handleStateChanges( + immutable.Set( + PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, replica) + ), + OnlineReplica + ) } //6. Set AR to RAR in memory. //7. Send LeaderAndIsr request with a potential new leader (if current leader not in RAR) and // a new AR (using RAR) and same isr to every broker in RAR - moveReassignedPartitionLeaderIfRequired(topicAndPartition, reassignedPartitionContext) + moveReassignedPartitionLeaderIfRequired(topicAndPartition, reassignedPartitionContext.newReplicas) //8. replicas in OAR - RAR -> Offline (force those replicas out of isr) //9. replicas in OAR - RAR -> NonExistentReplica (force those replicas to be deleted) - stopOldReplicasOfReassignedPartition(topicAndPartition, reassignedPartitionContext, oldReplicas) + stopOldReplicasOfReassignedPartition(topicAndPartition, oldReplicas) //10. Update AR in ZK with RAR. updateAssignedReplicasForPartition(topicAndPartition, reassignedReplicas) //11. Update the /admin/reassign_partitions path in ZK to remove this partition. @@ -606,7 +617,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState 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)) + sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, immutable.Set(topicAndPartition)) // signal delete topic thread if reassignment for some partitions belonging to topics being deleted just completed deleteTopicManager.resumeDeletionForTopics(Set(topicAndPartition.topic)) } @@ -614,23 +625,23 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState private def watchIsrChangesForReassignedPartition(topic: String, partition: Int, - reassignedPartitionContext: ReassignedPartitionsContext) { + reassignedPartitionContext: ReassignedPartitionsContext): Unit = { val reassignedReplicas = reassignedPartitionContext.newReplicas val isrChangeListener = new ReassignedPartitionsIsrChangeListener(this, topic, partition, reassignedReplicas.toSet) +// TODO: Mutating state here! reassignedPartitionContext.isrChangeListener = isrChangeListener // register listener on the leader and isr path to wait until they catch up with the current leader zkUtils.zkClient.subscribeDataChanges(getTopicPartitionLeaderAndIsrPath(topic, partition), isrChangeListener) } def initiateReassignReplicasForTopicPartition(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext) { + reassignedPartitionContext: ReassignedPartitionsContext): Unit = { val newReplicas = reassignedPartitionContext.newReplicas val topic = topicAndPartition.topic val partition = topicAndPartition.partition try { - val assignedReplicasOpt = controllerContext.partitionReplicaAssignment.get(topicAndPartition) - assignedReplicasOpt match { + controllerContext.partitionReplicaAssignment.get(topicAndPartition) match { case Some(assignedReplicas) => if (assignedReplicas == newReplicas) { throw new KafkaException("Partition %s to be reassigned is already assigned to replicas".format(topicAndPartition) + @@ -644,11 +655,10 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) onPartitionReassignment(topicAndPartition, reassignedPartitionContext) } - case None => throw new KafkaException("Attempt to reassign partition %s that doesn't exist" - .format(topicAndPartition)) + case None => throw new KafkaException(s"Attempt to reassign partition $partition that doesn't exist") } } catch { - case e: Throwable => error("Error completing reassignment of partition %s".format(topicAndPartition), e) + case e: Throwable => error(s"Error completing reassignment of partition $partition", e) // remove the partition from the admin path to unblock the admin client removePartitionFromReassignedPartitions(topicAndPartition) } @@ -775,21 +785,19 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState private def initializePartitionReassignment() { // read the partitions being reassigned from zookeeper path /admin/reassign_partitions val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned() - // 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 + val reassignedPartitions = partitionsBeingReassigned.filter { case (tp, ctx) => + // check if topic was deleted (None) or partition assignment already completed (Some with newReplicas) + controllerContext.partitionReplicaAssignment + .get(tp) + .forall(_ == ctx.newReplicas) }.keys reassignedPartitions.foreach(p => removePartitionFromReassignedPartitions(p)) - var partitionsToReassign: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap - partitionsToReassign ++= partitionsBeingReassigned - partitionsToReassign --= reassignedPartitions + val partitionsToReassign = partitionsBeingReassigned -- reassignedPartitions +// TODO: Mutation here! 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())) + info(s"Partitions being reassigned: $partitionsBeingReassigned") + info(s"Partitions already reassigned: $reassignedPartitions") + info(s"Resuming reassignment of partitions: $partitionsToReassign") } private def initializeTopicDeletion() { @@ -837,36 +845,34 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState } private def moveReassignedPartitionLeaderIfRequired(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext) { - val reassignedReplicas = reassignedPartitionContext.newReplicas + reassignedReplicas: Seq[Int]) { val currentLeader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader // change the assigned replica list to just the reassigned replicas in the cache so it gets sent out on the LeaderAndIsr // request to the current or new leader. This will prevent it from adding the old replicas to the ISR val oldAndNewReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) controllerContext.partitionReplicaAssignment.put(topicAndPartition, reassignedReplicas) - if(!reassignedPartitionContext.newReplicas.contains(currentLeader)) { - info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + - "is not in the new list of replicas %s. Re-electing leader".format(reassignedReplicas.mkString(","))) + if(!reassignedReplicas.contains(currentLeader)) { + info(s"Leader $currentLeader for partition $topicAndPartition being reassigned, is not in the new list of " + + s"replicas ${reassignedReplicas.mkString(",")}. Re-electing leader") // move the leader to one of the alive and caught up new replicas partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, reassignedPartitionLeaderSelector) } else { // check if the leader is alive or not if (controllerContext.liveBrokerIds.contains(currentLeader)) { - info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + - "is already in the new list of replicas %s and is alive".format(reassignedReplicas.mkString(","))) + info(s"Leader $currentLeader for partition $topicAndPartition being reassigned, is already in the new list of " + + s"replicas ${reassignedReplicas.mkString(",")} and is alive") // shrink replication factor and update the leader epoch in zookeeper to use on the next LeaderAndIsrRequest updateLeaderEpochAndSendRequest(topicAndPartition, oldAndNewReplicas, reassignedReplicas) } else { - info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + - "is already in the new list of replicas %s but is dead".format(reassignedReplicas.mkString(","))) + info(s"Leader $currentLeader for partition $topicAndPartition being reassigned, is already in the new list of " + + s"replicas ${reassignedReplicas.mkString(",")} but is dead") partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, reassignedPartitionLeaderSelector) } } } private def stopOldReplicasOfReassignedPartition(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext, - oldReplicas: Set[Int]) { + oldReplicas: immutable.Set[Int]) { val topic = topicAndPartition.topic val partition = topicAndPartition.partition // first move the replica to offline state (the controller removes it from the ISR) @@ -881,7 +887,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState private def updateAssignedReplicasForPartition(topicAndPartition: TopicAndPartition, replicas: Seq[Int]) { - val partitionsAndReplicasForThisTopic = controllerContext.partitionReplicaAssignment.filter(_._1.topic.equals(topicAndPartition.topic)) + val partitionsAndReplicasForThisTopic = controllerContext.partitionReplicaAssignment.filter(_._1.topic == topicAndPartition.topic) partitionsAndReplicasForThisTopic.put(topicAndPartition, replicas) updateAssignedReplicasForPartition(topicAndPartition, partitionsAndReplicasForThisTopic) info("Updated assigned replicas for partition %s being reassigned to %s ".format(topicAndPartition, replicas.mkString(","))) @@ -890,12 +896,16 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState } private def startNewReplicasForReassignedPartition(topicAndPartition: TopicAndPartition, - reassignedPartitionContext: ReassignedPartitionsContext, newReplicas: Set[Int]) { // send the start replica request to the brokers in the reassigned replicas list that are not in the assigned // replicas list newReplicas.foreach { replica => - replicaStateMachine.handleStateChanges(Set(new PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, replica)), NewReplica) + replicaStateMachine.handleStateChanges( + immutable.Set( + PartitionAndReplica(topicAndPartition.topic, topicAndPartition.partition, replica) + ), + NewReplica + ) } } @@ -972,11 +982,14 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState } def removePartitionFromReassignedPartitions(topicAndPartition: TopicAndPartition) { - if(controllerContext.partitionsBeingReassigned.get(topicAndPartition).isDefined) { - // stop watching the ISR changes for this partition - zkUtils.zkClient.unsubscribeDataChanges(getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition), - controllerContext.partitionsBeingReassigned(topicAndPartition).isrChangeListener) - } + controllerContext.partitionsBeingReassigned + .get(topicAndPartition) + .foreach(ctx => { + // stop watching the ISR changes for this partition + val isrChangeListener = ctx.isrChangeListener + val path = getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition) + zkUtils.zkClient.unsubscribeDataChanges(path, isrChangeListener) + }) // read the current list of reassigned partitions from zookeeper val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned() // remove this partition from that list @@ -1023,7 +1036,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState * * @param brokers The brokers that the update metadata request should be sent to */ - def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicAndPartition] = Set.empty[TopicAndPartition]) { + def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: immutable.Set[TopicAndPartition] = immutable.Set.empty[TopicAndPartition]) { try { brokerRequestBatch.newBatch() brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) @@ -1081,8 +1094,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState newIsr = leaderAndIsr.isr } - val newLeaderAndIsr = new LeaderAndIsr(newLeader, leaderAndIsr.leaderEpoch + 1, - newIsr, leaderAndIsr.zkVersion + 1) + val newLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(newLeader, newIsr) // update the new leadership decision in zookeeper or retry val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) @@ -1133,8 +1145,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState "controller was elected with epoch %d. Aborting state change by this controller".format(controllerEpoch)) // increment the leader epoch even if there are no leader or isr changes to allow the leader to cache the expanded // assigned replica list - val newLeaderAndIsr = new LeaderAndIsr(leaderAndIsr.leader, leaderAndIsr.leaderEpoch + 1, - leaderAndIsr.isr, leaderAndIsr.zkVersion + 1) + val newLeaderAndIsr = leaderAndIsr.newEpochAndZkVersion // update the new leadership decision in zookeeper or retry val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) @@ -1257,31 +1268,31 @@ class PartitionsReassignedListener(protected val controller: KafkaController) ex * @throws Exception On any error. */ @throws[Exception] - def doHandleDataChange(dataPath: String, data: AnyRef) { - debug("Partitions reassigned listener fired for path %s. Record partitions to be reassigned %s" - .format(dataPath, data)) + def doHandleDataChange(dataPath: String, data: AnyRef): Unit = { + debug(s"Partitions reassigned listener fired for path $dataPath. Record partitions to be reassigned $data") val partitionsReassignmentData = ZkUtils.parsePartitionReassignmentData(data.toString) val partitionsToBeReassigned = inLock(controllerContext.controllerLock) { partitionsReassignmentData.filterNot(p => controllerContext.partitionsBeingReassigned.contains(p._1)) } - partitionsToBeReassigned.foreach { partitionToBeReassigned => + partitionsToBeReassigned.foreach { case (tp @ TopicAndPartition(topic, partition), newReplicas) => 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) + if (controller.deleteTopicManager.isTopicQueuedUpForDeletion(topic)) { + error(s"Skipping reassignment of partition $partition for topic $topic since it is currently being deleted") + controller.removePartitionFromReassignedPartitions(tp) } else { - val context = new ReassignedPartitionsContext(partitionToBeReassigned._2) - controller.initiateReassignReplicasForTopicPartition(partitionToBeReassigned._1, context) + val context = ReassignedPartitionsContext(newReplicas) + controller.initiateReassignReplicasForTopicPartition(tp, context) } } } } - def doHandleDataDeleted(dataPath: String) {} + def doHandleDataDeleted(dataPath: String): Unit = {} } -class ReassignedPartitionsIsrChangeListener(protected val controller: KafkaController, topic: String, partition: Int, +class ReassignedPartitionsIsrChangeListener(protected val controller: KafkaController, + topic: String, + partition: Int, reassignedReplicas: Set[Int]) extends ControllerZkDataListener { private val zkUtils = controller.controllerContext.zkUtils private val controllerContext = controller.controllerContext @@ -1297,8 +1308,9 @@ class ReassignedPartitionsIsrChangeListener(protected val controller: KafkaContr val topicAndPartition = TopicAndPartition(topic, partition) try { // check if this partition is still being reassigned or not - controllerContext.partitionsBeingReassigned.get(topicAndPartition) match { - case Some(reassignedPartitionContext) => + controllerContext.partitionsBeingReassigned + .get(topicAndPartition) + .foreach( reassignedPartitionContext => { // need to re-read leader and isr from zookeeper since the zkclient callback doesn't return the Stat object val newLeaderAndIsrOpt = zkUtils.getLeaderAndIsrForPartition(topic, partition) newLeaderAndIsrOpt match { @@ -1319,8 +1331,7 @@ class ReassignedPartitionsIsrChangeListener(protected val controller: KafkaContr case None => error("Error handling reassignment of partition %s to replicas %s as it was never created" .format(topicAndPartition, reassignedReplicas.mkString(","))) } - case None => - } + }) } catch { case e: Throwable => error("Error while handling partition reassignment", e) } @@ -1375,7 +1386,7 @@ class IsrChangeNotificationListener(protected val controller: KafkaController) e val isrChanges = m.asInstanceOf[Map[String, Any]] val topicAndPartitionList = isrChanges("partitions").asInstanceOf[List[Any]] topicAndPartitionList.foreach { - case tp => + tp => val topicAndPartition = tp.asInstanceOf[Map[String, Any]] val topic = topicAndPartition("topic").asInstanceOf[String] val partition = topicAndPartition("partition").asInstanceOf[Int] @@ -1433,7 +1444,7 @@ class PreferredReplicaElectionListener(protected val controller: KafkaController } //TODO: Very dangerous, if put into map as key and later any var will change, magic will happen!!! -case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty, +case class ReassignedPartitionsContext(newReplicas: Seq[Int], var isrChangeListener: ReassignedPartitionsIsrChangeListener = null) case class PartitionAndReplica(topic: String, partition: Int, replica: Int) { diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 81719141ca0db..72978a39c94cc 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -52,11 +52,9 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { controllerContext.partitionReplicaAssignment.get(topicAndPartition) match { - case Some(assignedReplicas) => + case Some(assignedReplicas) => { 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 = if (liveBrokersInIsr.isEmpty) { // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration @@ -78,17 +76,19 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi 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, liveAssignedReplicas.mkString(","))) - new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1) + currentLeaderAndIsr.newLeaderAndIsr(newLeader, List(newLeader)) } } else { val liveReplicasInIsr = liveAssignedReplicas.filter(r => liveBrokersInIsr.contains(r)) val newLeader = liveReplicasInIsr.head debug("Some broker in ISR is alive for %s. Select %d from ISR %s to be the leader." .format(topicAndPartition, newLeader, liveBrokersInIsr.mkString(","))) - new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1) + currentLeaderAndIsr.newLeaderAndIsr(newLeader, liveBrokersInIsr) } - info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition)) + + info(s"Selected new leader and ISR $newLeaderAndIsr for offline partition $topicAndPartition") (newLeaderAndIsr, liveAssignedReplicas) + } case None => throw new NoReplicaOnlineException("Partition %s doesn't have replicas assigned to it".format(topicAndPartition)) } @@ -106,25 +106,29 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex /** * The reassigned replicas are already in the ISR when selectLeader is called. */ - def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { - val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned(topicAndPartition).newReplicas - val currentLeaderEpoch = currentLeaderAndIsr.leaderEpoch - val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion - val aliveReassignedInSyncReplicas = reassignedInSyncReplicas.filter(r => controllerContext.liveBrokerIds.contains(r) && - currentLeaderAndIsr.isr.contains(r)) + def selectLeader(topicAndPartition: TopicAndPartition, + currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { + val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned + .get(topicAndPartition) + .map(_.newReplicas) + .getOrElse(Seq.empty[Int]) + + val aliveReassignedInSyncReplicas = reassignedInSyncReplicas.filter(r => + controllerContext.liveBrokerIds.contains(r) && currentLeaderAndIsr.isr.contains(r) + ) + val newLeaderOpt = aliveReassignedInSyncReplicas.headOption newLeaderOpt match { - case Some(newLeader) => (new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, currentLeaderAndIsr.isr, - currentLeaderIsrZkPathVersion + 1), reassignedInSyncReplicas) + case Some(newLeader) => + (currentLeaderAndIsr.newLeader(newLeader), reassignedInSyncReplicas) + case None => - reassignedInSyncReplicas.size match { - case 0 => - throw new NoReplicaOnlineException("List of reassigned replicas for partition " + - " %s is empty. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) - case _ => - 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)) + val exceptionText = if (reassignedInSyncReplicas.isEmpty) { + s"List of reassigned replicas for partition $topicAndPartition is empty. Current leader and ISR: [$currentLeaderAndIsr]" + } else { + s"None of the reassigned replicas for partition $topicAndPartition are in-sync with the leader. Current leader and ISR: [$currentLeaderAndIsr]" } + throw new NoReplicaOnlineException(exceptionText) } } } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index c0b94b17cac12..c0cfe2f64c1c6 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -280,8 +280,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { debug("Live assigned replicas for partition %s are: [%s]".format(topicAndPartition, liveAssignedReplicas)) // make the first replica in the list of assigned replicas, the leader val leader = liveAssignedReplicas.head - val leaderIsrAndControllerEpoch = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader, liveAssignedReplicas.toList), - controller.epoch) + val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch( + LeaderAndIsr(leader, liveAssignedReplicas.toList), + controller.epoch + ) debug("Initializing leader and isr for partition %s to %s".format(topicAndPartition, leaderIsrAndControllerEpoch)) try { zkUtils.createPersistentPath( @@ -420,7 +422,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val deletedTopics = controllerContext.allTopics -- currentChildren controllerContext.allTopics = currentChildren - val addedPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(newTopics.toSeq) + val addedPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(newTopics.toSeq).toMap controllerContext.partitionReplicaAssignment = controllerContext.partitionReplicaAssignment.filter(p => !deletedTopics.contains(p._1.topic)) controllerContext.partitionReplicaAssignment.++=(addedPartitionReplicaAssignment) @@ -497,9 +499,10 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { inLock(controllerContext.controllerLock) { try { info(s"Partition modification triggered $data for path $dataPath") - val partitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(List(topic)) - val partitionsToBeAdded = partitionReplicaAssignment.filter(p => - !controllerContext.partitionReplicaAssignment.contains(p._1)) + val partitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(List(topic)).toMap + val partitionsToBeAdded = partitionReplicaAssignment.filter( + p => !controllerContext.partitionReplicaAssignment.contains(p._1) + ) 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)) @@ -521,6 +524,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } } +//TODO: state is not used anywhere, can we delete it? sealed trait PartitionState { def state: Byte } case object NewPartition extends PartitionState { val state: Byte = 0 } case object OnlinePartition extends PartitionState { val state: Byte = 1 } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index b106b0101eba2..6b256537d3469 100755 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -103,8 +103,9 @@ 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, - callbacks: Callbacks = (new CallbackBuilder).build) { + def handleStateChanges(replicas: immutable.Set[PartitionAndReplica], + targetState: ReplicaState, + callbacks: Callbacks = (new CallbackBuilder).build): Unit = { if(replicas.nonEmpty) { info("Invoking state change to %s for replicas %s".format(targetState, replicas.mkString(","))) try { @@ -152,7 +153,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * @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(partitionAndReplica: PartitionAndReplica, targetState: ReplicaState, + def handleStateChange(partitionAndReplica: PartitionAndReplica, + targetState: ReplicaState, callbacks: Callbacks) { val topic = partitionAndReplica.topic val partition = partitionAndReplica.partition @@ -289,8 +291,11 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { 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 replicasInState(topic: String, state: ReplicaState): immutable.Set[PartitionAndReplica] = { + replicaState + .filter(r => r._1.topic.equals(topic) && r._2 == state) + .keySet + .toSet } def isAnyReplicaInState(topic: String, state: ReplicaState): Boolean = { diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index fadc736742fdc..8b68ab63816b9 100755 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -169,7 +169,7 @@ class TopicDeletionManager(controller: KafkaController, * 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]) { + def failReplicaDeletion(replicas: scala.collection.immutable.Set[PartitionAndReplica]) { if(isDeleteTopicEnabled) { val replicasThatFailedToDelete = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) if(replicasThatFailedToDelete.nonEmpty) { @@ -256,9 +256,9 @@ class TopicDeletionManager(controller: KafkaController, * 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]) { + private def completeReplicaDeletion(replicas: scala.collection.immutable.Set[PartitionAndReplica]): Unit = { val successfullyDeletedReplicas = replicas.filter(r => isTopicQueuedUpForDeletion(r.topic)) - debug("Deletion successfully completed for replicas %s".format(successfullyDeletedReplicas.mkString(","))) + debug(s"Deletion successfully completed for replicas ${successfullyDeletedReplicas.mkString(",")}") controller.replicaStateMachine.handleStateChanges(successfullyDeletedReplicas, ReplicaDeletionSuccessful) resumeTopicDeletionThread() } @@ -315,14 +315,14 @@ class TopicDeletionManager(controller: KafkaController, * {@link LeaderAndIsr#LeaderDuringDelete}. This lets each broker know that this topic is being deleted and can be * removed from their caches. */ - private def onTopicDeletion(topics: Set[String]) { + private def onTopicDeletion(topics: scala.collection.immutable.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 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).keySet) + onPartitionDeletion(partitionReplicaAssignmentByTopic(topic).keySet.toSet) } } @@ -342,7 +342,7 @@ class TopicDeletionManager(controller: KafkaController, * 2. Move all alive replicas to ReplicaDeletionStarted state so they can be deleted successfully *@param replicasForTopicsToBeDeleted */ - private def startReplicaDeletion(replicasForTopicsToBeDeleted: Set[PartitionAndReplica]) { + private def startReplicaDeletion(replicasForTopicsToBeDeleted: scala.collection.immutable.Set[PartitionAndReplica]) { replicasForTopicsToBeDeleted.groupBy(_.topic).keys.foreach { topic => val aliveReplicasForTopic = controllerContext.allLiveReplicas().filter(p => p.topic == topic) val deadReplicasForTopic = replicasForTopicsToBeDeleted -- aliveReplicasForTopic @@ -373,8 +373,8 @@ class TopicDeletionManager(controller: KafkaController, * 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(","))) + private def onPartitionDeletion(partitionsToBeDeleted: scala.collection.immutable.Set[TopicAndPartition]) { + info(s"Partition deletion callback for ${partitionsToBeDeleted.mkString(",")}") val replicasPerPartition = controllerContext.replicasForPartition(partitionsToBeDeleted) startReplicaDeletion(replicasPerPartition) } @@ -382,7 +382,7 @@ class TopicDeletionManager(controller: KafkaController, private def deleteTopicStopReplicaCallback(stopReplicaResponseObj: AbstractResponse, replicaId: Int) { val stopReplicaResponse = stopReplicaResponseObj.asInstanceOf[StopReplicaResponse] debug("Delete topic callback invoked for %s".format(stopReplicaResponse)) - val responseMap = stopReplicaResponse.responses.asScala + val responseMap = stopReplicaResponse.responses.asScala.toMap val partitionsInError = if (stopReplicaResponse.error != Errors.NONE) responseMap.keySet else responseMap.filter { case (_, error) => error != Errors.NONE }.keySet @@ -407,7 +407,7 @@ class TopicDeletionManager(controller: KafkaController, return inLock(controllerContext.controllerLock) { - val topicsQueuedForDeletion = Set.empty[String] ++ topicsToBeDeleted + val topicsQueuedForDeletion = topicsToBeDeleted.toSet if(topicsQueuedForDeletion.nonEmpty) info("Handling deletion for topics " + topicsQueuedForDeletion.mkString(",")) @@ -438,11 +438,11 @@ class TopicDeletionManager(controller: KafkaController, } // Try delete topic if it is eligible for deletion. if(isTopicEligibleForDeletion(topic)) { - info("Deletion of topic %s (re)started".format(topic)) + info(s"Deletion of topic $topic (re)started") // topic deletion will be kicked off - onTopicDeletion(Set(topic)) + onTopicDeletion(scala.collection.immutable.Set(topic)) } else if(isTopicIneligibleForDeletion(topic)) { - info("Not retrying deletion of topic %s at this time since it is marked ineligible for deletion".format(topic)) + info(s"Not retrying deletion of topic $topic at this time since it is marked ineligible for deletion") } } } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 9a6090d3647d6..524c8f18c08f1 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -199,7 +199,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } private def partitionStateToPartitionStateInfo(partitionState: PartitionState): PartitionStateInfo = { - val leaderAndIsr = LeaderAndIsr(partitionState.leader, partitionState.leaderEpoch, partitionState.isr.asScala.map(_.toInt).toList, partitionState.zkVersion) + val leaderAndIsr = LeaderAndIsr(partitionState) val leaderInfo = LeaderIsrAndControllerEpoch(leaderAndIsr, partitionState.controllerEpoch) PartitionStateInfo(leaderInfo, partitionState.replicas.asScala.map(_.toInt)) } diff --git a/core/src/test/scala/unit/kafka/server/ControlledShutdownLeaderSelectorTest.scala b/core/src/test/scala/unit/kafka/server/ControlledShutdownLeaderSelectorTest.scala index 47a05efb638cb..4b907677dd4ab 100644 --- a/core/src/test/scala/unit/kafka/server/ControlledShutdownLeaderSelectorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControlledShutdownLeaderSelectorTest.scala @@ -45,7 +45,7 @@ class ControlledShutdownLeaderSelectorTest { controllerContext.partitionReplicaAssignment = mutable.Map(topicPartition -> assignment) val leaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - val firstLeaderAndIsr = new LeaderAndIsr(firstLeader, firstIsr) + val firstLeaderAndIsr = LeaderAndIsr(firstLeader, firstIsr) val (secondLeaderAndIsr, secondReplicas) = leaderSelector.selectLeader(topicPartition, firstLeaderAndIsr) assertEquals(preferredReplicaId, secondLeaderAndIsr.leader) diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 4eeb51529c0ca..ff976474ab978 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -144,8 +144,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { val staleControllerEpoch = 0 val partitionStates = Map( new TopicPartition(topic, partitionId) -> new PartitionState(2, brokerId2, LeaderAndIsr.initialLeaderEpoch, - Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, - Set(0, 1).map(Integer.valueOf).asJava) + Seq[Integer](brokerId1, brokerId2).asJava, LeaderAndIsr.initialZKVersion, Set[Integer](0, 1).asJava) ) val requestBuilder = new LeaderAndIsrRequest.Builder( controllerId, staleControllerEpoch, partitionStates.asJava, nodes.toSet.asJava) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 4f6a204b5d108..1e7385b2bd8a6 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -693,7 +693,7 @@ object TestUtils extends Logging { val currentLeaderAndIsrOpt = zkUtils.getLeaderAndIsrForPartition(topic, partition) var newLeaderAndIsr: LeaderAndIsr = null if(currentLeaderAndIsrOpt.isEmpty) - newLeaderAndIsr = new LeaderAndIsr(leader, List(leader)) + newLeaderAndIsr = LeaderAndIsr(leader, List(leader)) else{ newLeaderAndIsr = currentLeaderAndIsrOpt.get newLeaderAndIsr.leader = leader From 0b5b6049c555050bae5f66fcbd23caeb330546ed Mon Sep 17 00:00:00 2001 From: jozi-k Date: Mon, 27 Feb 2017 15:25:39 +0100 Subject: [PATCH 3/3] Make LeaderAndIsr immutable case class. --- .../main/scala/kafka/admin/AdminUtils.scala | 10 +- .../main/scala/kafka/api/LeaderAndIsr.scala | 22 +-- .../main/scala/kafka/cluster/Partition.scala | 6 +- .../kafka/controller/KafkaController.scala | 63 +++--- .../controller/PartitionLeaderSelector.scala | 67 ++++--- .../controller/PartitionStateMachine.scala | 5 +- .../coordinator/GroupMetadataManager.scala | 12 +- .../src/main/scala/kafka/log/LogManager.scala | 30 +-- .../kafka/server/AbstractFetcherManager.scala | 3 +- .../scala/kafka/server/ReplicaManager.scala | 185 ++++++++---------- .../kafka/tools/ConsumerOffsetChecker.scala | 7 +- core/src/main/scala/kafka/utils/Pool.scala | 3 +- .../scala/kafka/utils/ReplicationUtils.scala | 70 ++++--- core/src/main/scala/kafka/utils/ZkUtils.scala | 41 ++-- .../GroupCoordinatorResponseTest.scala | 3 +- .../GroupMetadataManagerTest.scala | 4 +- .../kafka/utils/ReplicationUtilsTest.scala | 10 +- .../scala/unit/kafka/utils/TestUtils.scala | 34 ++-- 18 files changed, 276 insertions(+), 299 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 65ac91cf361b4..ba766d59625ef 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -658,12 +658,10 @@ object AdminUtils extends Logging with AdminUtilities { if (zkUtils.pathExists(getTopicPath(topic))) { val topicPartitionAssignment = zkUtils.getPartitionAssignmentForTopics(List(topic))(topic) val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) - val partitionMetadata = sortedPartitions.map { partitionMap => - val partition = partitionMap._1 - val replicas = partitionMap._2 + val partitionMetadata = sortedPartitions.map { case (partition, replicas) => val inSyncReplicas = zkUtils.getInSyncReplicasForPartition(topic, partition) val leader = zkUtils.getLeaderForPartition(topic, partition) - debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader) + debug(s"replicas = $replicas, in sync replicas = $inSyncReplicas, leader = $leader") var leaderInfo: Node = Node.noNode() var replicaInfo: Seq[Node] = Nil @@ -678,12 +676,14 @@ object AdminUtils extends Logging with AdminUtilities { } case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) } + try { replicaInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, replicas).map(_.getNode(listenerName)) isrInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, inSyncReplicas).map(_.getNode(listenerName)) } catch { case e: Throwable => throw new ReplicaNotAvailableException(e) } + if (replicaInfo.size < replicas.size) throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) @@ -691,6 +691,7 @@ object AdminUtils extends Logging with AdminUtilities { throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + inSyncReplicas.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) new MetadataResponse.PartitionMetadata(Errors.NONE, partition, leaderInfo, replicaInfo.asJava, isrInfo.asJava) + } catch { case e: Throwable => debug("Error while fetching metadata for partition [%s,%d]".format(topic, partition), e) @@ -707,6 +708,7 @@ object AdminUtils extends Logging with AdminUtilities { private def getBrokerInfoFromCache(zkUtils: ZkUtils, cachedBrokerInfo: scala.collection.mutable.Map[Int, Broker], brokerIds: Seq[Int]): Seq[Broker] = { +// TODO: This buffer is ignored, can be deleted? var failedBrokerIds: ListBuffer[Int] = new ListBuffer() val brokerMetadata = brokerIds.map { id => val optionalBrokerInfo = cachedBrokerInfo.get(id) diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala index 6930e7316b490..504b138b5590e 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala @@ -32,22 +32,24 @@ object LeaderAndIsr { def apply(leader: Int, isr: List[Int]): LeaderAndIsr = LeaderAndIsr(leader, initialLeaderEpoch, isr, initialZKVersion) - def apply(partitionState: PartitionState): LeaderAndIsr = { + def apply(partitionState: PartitionState): LeaderAndIsr = LeaderAndIsr( partitionState.leader, partitionState.leaderEpoch, partitionState.isr.asScala.map(_.toInt).toList, partitionState.zkVersion ) - } } -//TODO: Do not use vars!!!, change isr to Set[Int] -case class LeaderAndIsr(var leader: Int, - var leaderEpoch: Int, - var isr: List[Int], - var zkVersion: Int) { - def newLeader(newLeader: Int) = LeaderAndIsr(newLeader, leaderEpoch + 1, isr, zkVersion + 1) +//TODO: Can we change isr to Set[Int]? +case class LeaderAndIsr(leader: Int, + leaderEpoch: Int, + isr: List[Int], + zkVersion: Int) { + def newLeader(leader: Int) = LeaderAndIsr(leader, leaderEpoch + 1, isr, zkVersion + 1) + +// TODO: Remove this method. + def newZkVersion(zkVersion: Int) = this.copy(zkVersion = zkVersion) def newLeaderAndIsr(newLeader: Int, newIsr: List[Int]) = LeaderAndIsr(newLeader, leaderEpoch + 1, newIsr, zkVersion + 1) @@ -61,12 +63,10 @@ case class LeaderAndIsr(var leader: Int, case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Set[Int]) { - def replicationFactor = allReplicas.size - override def toString: String = { val partitionStateInfo = new StringBuilder partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString) - partitionStateInfo.append(",ReplicationFactor:" + replicationFactor + ")") + partitionStateInfo.append(",ReplicationFactor:" + allReplicas.size + ")") partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")") partitionStateInfo.toString() } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 9e51faea85003..4eac45f50cd04 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -469,13 +469,13 @@ class Partition(val topic: String, private def updateIsr(newIsr: Set[Replica]) { val newLeaderAndIsr = LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) - val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partitionId, + val (updateSucceeded, newZkVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partitionId, newLeaderAndIsr, controllerEpoch, zkVersion) - if(updateSucceeded) { + if (updateSucceeded) { replicaManager.recordIsrChange(topicPartition) inSyncReplicas = newIsr - zkVersion = newVersion + zkVersion = newZkVersion trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion)) } else { info("Cached zkVersion [%d] not equal to that in zookeeper, skip updating ISR".format(zkVersion)) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 98e23af524705..c59d4969aa7e1 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -793,7 +793,6 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState }.keys reassignedPartitions.foreach(p => removePartitionFromReassignedPartitions(p)) val partitionsToReassign = partitionsBeingReassigned -- reassignedPartitions -// TODO: Mutation here! controllerContext.partitionsBeingReassigned ++= partitionsToReassign info(s"Partitions being reassigned: $partitionsBeingReassigned") info(s"Partitions already reassigned: $reassignedPartitions") @@ -1099,11 +1098,11 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) - newLeaderAndIsr.zkVersion = newVersion - finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) + val newNewLeaderAndIsr = newLeaderAndIsr.newZkVersion(newVersion) + finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newNewLeaderAndIsr, epoch)) controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get) if (updateSucceeded) - info("New leader and ISR for partition %s is %s".format(topicAndPartition, newLeaderAndIsr.toString())) + info(s"New leader and ISR for partition $topicAndPartition is $newNewLeaderAndIsr") updateSucceeded } else { warn("Cannot remove replica %d from ISR of partition %s since it is not in the ISR. Leader = %d ; ISR = %s" @@ -1147,17 +1146,17 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState // assigned replica list val newLeaderAndIsr = leaderAndIsr.newEpochAndZkVersion // update the new leadership decision in zookeeper or retry - val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, + val (updateSucceeded, newZkVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) - newLeaderAndIsr.zkVersion = newVersion - finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) + val newNewLeaderAndIsr = newLeaderAndIsr.newZkVersion(newZkVersion) + finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newNewLeaderAndIsr, epoch)) if (updateSucceeded) - info("Updated leader epoch for partition %s to %d".format(topicAndPartition, newLeaderAndIsr.leaderEpoch)) + info(s"Updated leader epoch for partition $topicAndPartition to ${newNewLeaderAndIsr.leaderEpoch}") updateSucceeded case None => - throw new IllegalStateException(("Cannot update leader epoch for partition %s as leaderAndIsr path is empty. " + - "This could mean we somehow tried to reassign a partition that doesn't exist").format(topicAndPartition)) + throw new IllegalStateException(s"Cannot update leader epoch for partition $topicAndPartition as leaderAndIsr " + + "path is empty. This could mean we somehow tried to reassign a partition that doesn't exist") true } } @@ -1374,32 +1373,28 @@ class IsrChangeNotificationListener(protected val controller: KafkaController) e controller.sendUpdateMetadataRequest(liveBrokers, topicAndPartitions) } - private def getTopicAndPartition(child: String): Set[TopicAndPartition] = { + private def getTopicAndPartition(child: String): immutable.Set[TopicAndPartition] = { val changeZnode: String = ZkUtils.IsrChangeNotificationPath + "/" + child val (jsonOpt, _) = controller.controllerContext.zkUtils.readDataMaybeNull(changeZnode) - if (jsonOpt.isDefined) { - val json = Json.parseFull(jsonOpt.get) - - json match { - case Some(m) => - val topicAndPartitions: mutable.Set[TopicAndPartition] = new mutable.HashSet[TopicAndPartition]() - val isrChanges = m.asInstanceOf[Map[String, Any]] - val topicAndPartitionList = isrChanges("partitions").asInstanceOf[List[Any]] - topicAndPartitionList.foreach { - tp => - val topicAndPartition = tp.asInstanceOf[Map[String, Any]] - val topic = topicAndPartition("topic").asInstanceOf[String] - val partition = topicAndPartition("partition").asInstanceOf[Int] - topicAndPartitions += TopicAndPartition(topic, partition) - } - topicAndPartitions - case None => - error("Invalid topic and partition JSON: " + jsonOpt.get + " in ZK: " + changeZnode) - Set.empty + jsonOpt.toSet[String] + .flatMap { json => + Json.parseFull(json) match { + case Some(m) => + val isrChanges = m.asInstanceOf[Map[String, Any]] + val topicAndPartitionList = isrChanges("partitions").asInstanceOf[List[Any]] + topicAndPartitionList.toSet[Any].map { + tp => + val topicAndPartition = tp.asInstanceOf[Map[String, Any]] + val topic = topicAndPartition("topic").asInstanceOf[String] + val partition = topicAndPartition("partition").asInstanceOf[Int] + TopicAndPartition(topic, partition) + } + case None => +// TODO: If we get rid of this message, whole method can be rewritten via for comprehension + error(s"Invalid topic and partition JSON: $json in ZK: $changeZnode") + immutable.Set.empty[TopicAndPartition] + } } - } else { - Set.empty - } } } @@ -1443,7 +1438,7 @@ class PreferredReplicaElectionListener(protected val controller: KafkaController def doHandleDataDeleted(dataPath: String) {} } -//TODO: Very dangerous, if put into map as key and later any var will change, magic will happen!!! +//TODO: Change to val, can newReplicas be Set? case class ReassignedPartitionsContext(newReplicas: Seq[Int], var isrChangeListener: ReassignedPartitionsIsrChangeListener = null) diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 72978a39c94cc..1bb59569c1cb2 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -100,14 +100,16 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi * New isr = current isr * Replicas to receive LeaderAndIsr request = reassigned replicas */ -class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { - this.logIdent = "[ReassignedPartitionLeaderSelector]: " +class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) + extends PartitionLeaderSelector + with Logging { + logIdent = "[ReassignedPartitionLeaderSelector]: " /** * The reassigned replicas are already in the ISR when selectLeader is called. */ - def selectLeader(topicAndPartition: TopicAndPartition, - currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { + override def selectLeader(topicAndPartition: TopicAndPartition, + currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned .get(topicAndPartition) .map(_.newReplicas) @@ -138,28 +140,31 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex * New isr = current isr; * Replicas to receive LeaderAndIsr request = assigned replicas */ -class PreferredReplicaPartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector -with Logging { - this.logIdent = "[PreferredReplicaPartitionLeaderSelector]: " +class PreferredReplicaPartitionLeaderSelector(controllerContext: ControllerContext) + extends PartitionLeaderSelector + with Logging { - def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { + logIdent = "[PreferredReplicaPartitionLeaderSelector]: " + + override def selectLeader(topicAndPartition: TopicAndPartition, + currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) +// TODO: Possible NoSuchElementException val preferredReplica = assignedReplicas.head // check if preferred replica is the current leader val currentLeader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader if (currentLeader == preferredReplica) { - throw new LeaderElectionNotNeededException("Preferred replica %d is already the current leader for partition %s" - .format(preferredReplica, topicAndPartition)) + throw new LeaderElectionNotNeededException( + s"Preferred replica $preferredReplica is already the current leader for partition $topicAndPartition") } else { - info("Current leader %d for partition %s is not the preferred replica.".format(currentLeader, topicAndPartition) + + info(s"Current leader $currentLeader for partition $topicAndPartition is not the preferred replica." + " Triggering preferred replica leader election") // check if preferred replica is not the current leader and is alive and in the isr if (controllerContext.liveBrokerIds.contains(preferredReplica) && currentLeaderAndIsr.isr.contains(preferredReplica)) { - (new LeaderAndIsr(preferredReplica, currentLeaderAndIsr.leaderEpoch + 1, currentLeaderAndIsr.isr, - currentLeaderAndIsr.zkVersion + 1), assignedReplicas) + (currentLeaderAndIsr.newLeader(preferredReplica), assignedReplicas) } else { - throw new StateChangeFailedException("Preferred replica %d for partition ".format(preferredReplica) + - "%s is either not alive or not in the isr. Current leader and ISR: [%s]".format(topicAndPartition, currentLeaderAndIsr)) + throw new StateChangeFailedException(s"Preferred replica $preferredReplica for partition $topicAndPartition " + + s"is either not alive or not in the isr. Current leader and ISR: [$currentLeaderAndIsr]") } } } @@ -171,17 +176,13 @@ with Logging { * Replicas to receive LeaderAndIsr request = live assigned replicas */ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) - extends PartitionLeaderSelector - with Logging { - - this.logIdent = "[ControlledShutdownLeaderSelector]: " + extends PartitionLeaderSelector + with Logging { - def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { - val currentLeaderEpoch = currentLeaderAndIsr.leaderEpoch - val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion - - val currentLeader = currentLeaderAndIsr.leader + logIdent = "[ControlledShutdownLeaderSelector]: " + override def selectLeader(topicAndPartition: TopicAndPartition, + currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) val liveOrShuttingDownBrokerIds = controllerContext.liveOrShuttingDownBrokerIds val liveAssignedReplicas = assignedReplicas.filter(r => liveOrShuttingDownBrokerIds.contains(r)) @@ -189,11 +190,12 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) val newIsr = currentLeaderAndIsr.isr.filter(brokerId => !controllerContext.shuttingDownBrokerIds.contains(brokerId)) liveAssignedReplicas.find(newIsr.contains) match { case Some(newLeader) => - debug("Partition %s : current leader = %d, new leader = %d".format(topicAndPartition, currentLeader, newLeader)) - (LeaderAndIsr(newLeader, currentLeaderEpoch + 1, newIsr, currentLeaderIsrZkPathVersion + 1), liveAssignedReplicas) + debug(s"Partition $topicAndPartition: current leader = ${currentLeaderAndIsr.leader}, new leader = $newLeader") + (currentLeaderAndIsr.newLeaderAndIsr(newLeader, newIsr), liveAssignedReplicas) case None => - throw new StateChangeFailedException(("No other replicas in ISR %s for %s besides" + - " shutting down brokers %s").format(currentLeaderAndIsr.isr.mkString(","), topicAndPartition, controllerContext.shuttingDownBrokerIds.mkString(","))) + throw new StateChangeFailedException( + s"No other replicas in ISR ${currentLeaderAndIsr.isr.mkString(",")} for $topicAndPartition besides " + + s"shutting down brokers ${controllerContext.shuttingDownBrokerIds.mkString(",")}") } } } @@ -202,11 +204,14 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) * Essentially does nothing. Returns the current leader and ISR, and the current * set of replicas assigned to a given topic/partition. */ -class NoOpLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { +class NoOpLeaderSelector(controllerContext: ControllerContext) + extends PartitionLeaderSelector + with Logging { - this.logIdent = "[NoOpLeaderSelector]: " + logIdent = "[NoOpLeaderSelector]: " - def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { + def selectLeader(topicAndPartition: TopicAndPartition, + currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { warn("I should never have been asked to perform leader election, returning the current LeaderAndIsr and replica assignment.") (currentLeaderAndIsr, controllerContext.partitionReplicaAssignment(topicAndPartition)) } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index c0cfe2f64c1c6..de518ef4a3a0c 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -341,12 +341,11 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { val (leaderAndIsr, replicas) = leaderSelector.selectLeader(topicAndPartition, currentLeaderAndIsr) val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition, leaderAndIsr, controller.epoch, currentLeaderAndIsr.zkVersion) - newLeaderAndIsr = leaderAndIsr - newLeaderAndIsr.zkVersion = newVersion + newLeaderAndIsr = leaderAndIsr.newZkVersion(newVersion) zookeeperPathUpdateSucceeded = updateSucceeded replicasForThisPartition = replicas } - val newLeaderIsrAndControllerEpoch = new LeaderIsrAndControllerEpoch(newLeaderAndIsr, controller.epoch) + val newLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(newLeaderAndIsr, controller.epoch) // update the leader cache controllerContext.partitionLeadershipInfo.put(TopicAndPartition(topic, partition), newLeaderIsrAndControllerEpoch) stateChangeLogger.trace("Controller %d epoch %d elected leader %d for Offline partition %s" diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 8bbb506baa4c2..617957e60901a 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -643,13 +643,13 @@ class GroupMetadataManager(val brokerId: Int, * Gets the partition count of the offsets topic from ZooKeeper. * If the topic does not exist, the configured partition count is returned. */ - private def getOffsetsTopicPartitionCount = { + private def getOffsetsTopicPartitionCount: Int = { val topic = Topic.GroupMetadataTopicName - val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic)) - if (topicData(topic).nonEmpty) - topicData(topic).size - else - config.offsetsTopicNumPartitions + zkUtils.getPartitionAssignmentForTopics(Seq(topic)) + .get(topic) + .filter(_.nonEmpty) + .map(_.size) + .getOrElse(config.offsetsTopicNumPartitions) } /** diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 3f14937d8b847..f8ec1eda8713f 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -67,6 +67,7 @@ class LogManager(val logDirs: Array[File], loadLogs() // public, so we can access this from kafka.admin.DeleteTopicTest +// TODO: Make Option[LogCleaner] val cleaner: LogCleaner = if(cleanerConfig.enableCleaner) new LogCleaner(cleanerConfig, logDirs, logs, time = time) @@ -289,21 +290,22 @@ class LogManager(val logDirs: Array[File], * * @param partitionOffsets Partition logs that need to be truncated */ - def truncateTo(partitionOffsets: Map[TopicPartition, Long]) { - for ((topicPartition, truncateOffset) <- partitionOffsets) { - val log = logs.get(topicPartition) - // If the log does not exist, skip it - if (log != null) { - //May need to abort and pause the cleaning of the log, and resume after truncation is done. - val needToStopCleaner: Boolean = truncateOffset < log.activeSegment.baseOffset - if (needToStopCleaner && cleaner != null) - cleaner.abortAndPauseCleaning(topicPartition) - log.truncateTo(truncateOffset) - if (needToStopCleaner && cleaner != null) { - cleaner.maybeTruncateCheckpoint(log.dir.getParentFile, topicPartition, log.activeSegment.baseOffset) - cleaner.resumeCleaning(topicPartition) + def truncateTo(partitionOffsets: Map[TopicPartition, Long]): Unit = { + partitionOffsets.foreach { + case (topicPartition, truncateOffset) => + val log = logs.get(topicPartition) + // If the log does not exist, skip it + if (log != null) { + //May need to abort and pause the cleaning of the log, and resume after truncation is done. + val needToStopCleaner: Boolean = truncateOffset < log.activeSegment.baseOffset + if (needToStopCleaner && cleaner != null) + cleaner.abortAndPauseCleaning(topicPartition) + log.truncateTo(truncateOffset) + if (needToStopCleaner && cleaner != null) { + cleaner.maybeTruncateCheckpoint(log.dir.getParentFile, topicPartition, log.activeSegment.baseOffset) + cleaner.resumeCleaning(topicPartition) + } } - } } checkpointRecoveryPointOffsets() } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 87d09bf14950b..be42f159955e4 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -103,8 +103,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri def removeFetcherForPartitions(partitions: Set[TopicPartition]) { mapLock synchronized { - for (fetcher <- fetcherThreadMap.values) - fetcher.removePartitions(partitions) + fetcherThreadMap.valuesIterator.foreach(_.removePartitions(partitions)) } info(s"Removed fetcher for partitions ${partitions.mkString(",")}") } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 69f6d093e59e5..ab736d51ddd2c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -179,7 +179,7 @@ class ReplicaManager(val config: KafkaConfig, if (isrChangeSet.nonEmpty && (lastIsrChangeMs.get() + ReplicaManager.IsrChangePropagationBlackOut < now || lastIsrPropagationMs.get() + ReplicaManager.IsrChangePropagationInterval < now)) { - ReplicationUtils.propagateIsrChanges(zkUtils, isrChangeSet) + ReplicationUtils.propagateIsrChanges(zkUtils, isrChangeSet.toSet) isrChangeSet.clear() lastIsrPropagationMs.set(now) } @@ -641,7 +641,8 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(correlationId: Int,leaderAndISRRequest: LeaderAndIsrRequest, + def becomeLeaderOrFollower(correlationId: Int, + leaderAndISRRequest: LeaderAndIsrRequest, metadataCache: MetadataCache, onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): BecomeLeaderOrFollowerResult = { leaderAndISRRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) => @@ -650,8 +651,6 @@ class ReplicaManager(val config: KafkaConfig, leaderAndISRRequest.controllerId, leaderAndISRRequest.controllerEpoch, topicPartition.topic, topicPartition.partition)) } replicaStateChangeLock synchronized { -// TODO: Try to get rid of mutable map - val responseMap = new mutable.HashMap[TopicPartition, Errors] if (leaderAndISRRequest.controllerEpoch < controllerEpoch) { stateChangeLogger.warn( s"Broker $localBrokerId ignoring LeaderAndIsr request from controller ${leaderAndISRRequest.controllerId} " + @@ -663,24 +662,23 @@ class ReplicaManager(val config: KafkaConfig, controllerEpoch = leaderAndISRRequest.controllerEpoch // First check partition's leader epoch - val partitionStateMap = leaderAndISRRequest.partitionStates() + val (partitionStateMap, responseMap) = leaderAndISRRequest.partitionStates() .asScala - .foldLeft(Map.empty[Partition, PartitionState]) { - case (partitionAcc, (topicPartition, stateInfo)) => + .foldLeft((Map.empty[Partition, PartitionState], Map.empty[TopicPartition, Errors])) { + case ((partitionAcc, responseAcc), (topicPartition, stateInfo)) => val partition = getOrCreatePartition(topicPartition) val partitionLeaderEpoch = partition.getLeaderEpoch // If the leader epoch is valid record the epoch of the controller that made the leadership decision. // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path if (partitionLeaderEpoch < stateInfo.leaderEpoch) { if(stateInfo.replicas.contains(localBrokerId)) { - partitionAcc + (partition -> stateInfo) + (partitionAcc + (partition -> stateInfo), responseAcc + (topicPartition -> Errors.NONE)) } else { stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + "epoch %d for partition [%s,%d] as itself is not in assigned replica list %s") .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, topicPartition.topic, topicPartition.partition, stateInfo.replicas.asScala.mkString(","))) - responseMap.put(topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION) - partitionAcc + (partitionAcc, responseAcc + (topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION)) } } else { // Otherwise record the error code in response @@ -688,23 +686,15 @@ class ReplicaManager(val config: KafkaConfig, "epoch %d for partition [%s,%d] since its associated leader epoch %d is not higher than the current leader epoch %d") .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, topicPartition.topic, topicPartition.partition, stateInfo.leaderEpoch, partitionLeaderEpoch)) - responseMap.put(topicPartition, Errors.STALE_CONTROLLER_EPOCH) - partitionAcc + (partitionAcc, responseAcc + (topicPartition -> Errors.STALE_CONTROLLER_EPOCH)) } } - val partitionsTobeLeader = partitionStateMap.filter { case (_, stateInfo) => stateInfo.leader == localBrokerId} - val partitionsToBeFollower = partitionStateMap -- partitionsTobeLeader.keys + val (partitionsTobeLeader, partitionsToBeFollower) = + partitionStateMap.partition { case (_, stateInfo) => stateInfo.leader == localBrokerId} - val partitionsBecomeLeader = if (partitionsTobeLeader.nonEmpty) - makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, correlationId, responseMap) - else - Set.empty[Partition] - - val partitionsBecomeFollower = if (partitionsToBeFollower.nonEmpty) - makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, correlationId, responseMap, metadataCache) - else - Set.empty[Partition] + val partitionsBecomeLeader = makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, correlationId) + val partitionsBecomeFollower = makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, correlationId, metadataCache) // 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 @@ -715,7 +705,7 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.shutdownIdleFetcherThreads() onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower) - BecomeLeaderOrFollowerResult(responseMap.toMap, Errors.NONE) + BecomeLeaderOrFollowerResult(responseMap, Errors.NONE) } } } @@ -736,54 +726,52 @@ class ReplicaManager(val config: KafkaConfig, private def makeLeaders(controllerId: Int, epoch: Int, partitionState: immutable.Map[Partition, PartitionState], - correlationId: Int, - responseMap: mutable.Map[TopicPartition, Errors]): Set[Partition] = { - partitionState.keys.foreach { partition => - stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + - "starting the become-leader transition for partition %s") - .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) + correlationId: Int): immutable.Set[Partition] = { + partitionState.foreach { case (partition, _) => + stateChangeLogger.trace(s"Broker $localBrokerId handling LeaderAndIsr request correlationId $correlationId " + + s"from controller $controllerId epoch $epoch starting the become-leader transition for partition ${partition.topicPartition}") } - for (partition <- partitionState.keys) - responseMap.put(partition.topicPartition, Errors.NONE) - - val partitionsToMakeLeaders: mutable.Set[Partition] = mutable.Set() - - try { + val (leaders, notLeaders) = try { // First stop fetchers for all the partitions replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(_.topicPartition)) // Update the partition information to be the leader - partitionState.foreach{ case (partition, partitionStateInfo) => - if (partition.makeLeader(controllerId, partitionStateInfo, correlationId)) - partitionsToMakeLeaders += partition - else - stateChangeLogger.info(("Broker %d skipped the become-leader state change after marking its partition as leader with correlation id %d from " + - "controller %d epoch %d for partition %s since it is already the leader for the partition.") - .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) - } - partitionsToMakeLeaders.foreach { partition => - stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-leader request from controller " + - "%d epoch %d with correlation id %d for partition %s") - .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition)) +// TODO: Check if exception can be thrown in makeLeader + partitionState.partition { + case (partition, state) => partition.makeLeader(controllerId, state, correlationId) } } catch { - case e: Throwable => - partitionState.keys.foreach { partition => - val errorMsg = ("Error on broker %d while processing LeaderAndIsr request correlationId %d received from controller %d" + - " epoch %d for partition %s").format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition) - stateChangeLogger.error(errorMsg, e) - } - // Re-throw the exception for it to be caught in KafkaApis - throw e + case e: Throwable => + partitionState.foreach { case (partition, _) => + lazy val errorMsg = s"Error on broker $localBrokerId while processing LeaderAndIsr request correlationId $correlationId " + + s"received from controller $controllerId epoch $epoch for partition ${partition.topicPartition}" + stateChangeLogger.error(errorMsg, e) + } + // Re-throw the exception for it to be caught in KafkaApis + throw e + } + + notLeaders.foreach { case (partition, _) => + stateChangeLogger.info( + s"Broker $localBrokerId skipped the become-leader state change after marking its partition as leader with " + + s"correlation id $correlationId from controller $controllerId epoch $epoch for partition ${partition.topicPartition}" + + s"since it is already the leader for the partition." + ) + } + + leaders.foreach { case (partition, _) => + stateChangeLogger.trace( + s"Broker $localBrokerId stopped fetchers as part of become-leader request from controller $controllerId " + + s"epoch $epoch with correlation id $correlationId for partition ${partition.topicPartition}" + ) } - partitionState.keys.foreach { partition => - stateChangeLogger.trace(("Broker %d completed LeaderAndIsr request correlationId %d from controller %d epoch %d " + - "for the become-leader transition for partition %s") - .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) + partitionState.foreach { case (partition, _) => + stateChangeLogger.trace(s"Broker $localBrokerId completed LeaderAndIsr request correlationId $correlationId " + + s"from controller $controllerId epoch $epoch for the become-leader transition for partition ${partition.topicPartition}") } - partitionsToMakeLeaders + leaders.keySet } /* @@ -808,23 +796,17 @@ class ReplicaManager(val config: KafkaConfig, epoch: Int, partitionState: Map[Partition, PartitionState], correlationId: Int, - responseMap: mutable.Map[TopicPartition, Errors], - metadataCache: MetadataCache) : Set[Partition] = { - partitionState.keys.foreach { partition => - stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + - "starting the become-follower transition for partition %s") - .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) + metadataCache: MetadataCache) : immutable.Set[Partition] = { + partitionState.foreach { case (partition, _) => + stateChangeLogger.trace(s"Broker $localBrokerId handling LeaderAndIsr request correlationId $correlationId from " + + s"controller $controllerId epoch $epoch starting the become-follower transition for partition ${partition.topicPartition}") } - for (partition <- partitionState.keys) - responseMap.put(partition.topicPartition, Errors.NONE) - - val partitionsToMakeFollower: mutable.Set[Partition] = mutable.Set() + val partitionsToMakeFollower = mutable.Set.empty[Partition] try { - // TODO: Delete leaders from LeaderAndIsrRequest - partitionState.foreach{ case (partition, partitionStateInfo) => + partitionState.foreach { case (partition, partitionStateInfo) => val newLeaderBrokerId = partitionStateInfo.leader metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match { // Only change partition state when the leader is available @@ -851,14 +833,15 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(_.topicPartition).toSet) partitionsToMakeFollower.foreach { partition => - stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-follower request from controller " + - "%d epoch %d with correlation id %d for partition %s") - .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition)) + stateChangeLogger.trace(s"Broker $localBrokerId stopped fetchers as part of become-follower request from controller" + + s"$controllerId epoch $epoch with correlation id $correlationId for partition ${partition.topicPartition}") } - logManager.truncateTo(partitionsToMakeFollower.map { partition => - (partition.topicPartition, partition.getOrCreateReplica().highWatermark.messageOffset) - }.toMap) + logManager.truncateTo( + partitionsToMakeFollower.map { partition => + partition.topicPartition -> partition.getOrCreateReplica().highWatermark.messageOffset + }.toMap + ) partitionsToMakeFollower.foreach { partition => val topicPartitionOperationKey = new TopicPartitionOperationKey(partition.topicPartition) tryCompleteDelayedProduce(topicPartitionOperationKey) @@ -866,30 +849,35 @@ class ReplicaManager(val config: KafkaConfig, } partitionsToMakeFollower.foreach { partition => - stateChangeLogger.trace(("Broker %d truncated logs and checkpointed recovery boundaries for partition %s as part of " + - "become-follower request with correlation id %d from controller %d epoch %d").format(localBrokerId, - partition.topicPartition, correlationId, controllerId, epoch)) + stateChangeLogger.trace(s"Broker $localBrokerId truncated logs and checkpointed recovery boundaries for " + + s"partition ${partition.topicPartition} as part of become-follower request with correlation id $correlationId " + + s"from controller $controllerId epoch $epoch") } if (isShuttingDown.get()) { partitionsToMakeFollower.foreach { partition => - stateChangeLogger.trace(("Broker %d skipped the adding-fetcher step of the become-follower state change with correlation id %d from " + - "controller %d epoch %d for partition %s since it is shutting down").format(localBrokerId, correlationId, - controllerId, epoch, partition.topicPartition)) + stateChangeLogger.trace(s"Broker $localBrokerId skipped the adding-fetcher step of the become-follower state " + + s"change with correlation id $correlationId from controller $controllerId epoch $epoch for " + + s"partition ${partition.topicPartition} since it is shutting down") } - } - else { + } else { // we do not need to check if the leader exists again since this has been done at the beginning of this process - val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => - partition.topicPartition -> BrokerAndInitialOffset( - metadataCache.getAliveBrokers.find(_.id == partition.leaderReplicaIdOpt.get).get.getBrokerEndPoint(config.interBrokerListenerName), - partition.getReplica().get.logEndOffset.messageOffset)).toMap + val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map( + partition => { + val broker = metadataCache.getAliveBrokers + .find(_.id == partition.leaderReplicaIdOpt.get) + .get + .getBrokerEndPoint(config.interBrokerListenerName) + val offset = partition.getReplica().get.logEndOffset.messageOffset + partition.topicPartition -> BrokerAndInitialOffset(broker, offset) + } + ).toMap + replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) partitionsToMakeFollower.foreach { partition => - stateChangeLogger.trace(("Broker %d started fetcher to new leader as part of become-follower request from controller " + - "%d epoch %d with correlation id %d for partition %s") - .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition)) + stateChangeLogger.trace(s"Broker $localBrokerId started fetcher to new leader as part of become-follower request" + + s"from controller $controllerId epoch $epoch with correlation id $correlationId for partition ${partition.topicPartition}") } } } catch { @@ -901,13 +889,12 @@ class ReplicaManager(val config: KafkaConfig, throw e } - partitionState.keys.foreach { partition => - stateChangeLogger.trace(("Broker %d completed LeaderAndIsr request correlationId %d from controller %d epoch %d " + - "for the become-follower transition for partition %s") - .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) + partitionState.foreach { case (partition, _) => + stateChangeLogger.trace(s"Broker $localBrokerId completed LeaderAndIsr request correlationId $correlationId " + + s"from controller $controllerId epoch $epoch for the become-follower transition for partition ${partition.topicPartition}") } - partitionsToMakeFollower + partitionsToMakeFollower.toSet } private def maybeShrinkIsr(): Unit = { diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index b2699668a10fd..fd47651085bd3 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -38,7 +38,6 @@ object ConsumerOffsetChecker extends Logging { private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map() private val offsetMap: mutable.Map[TopicAndPartition, Long] = mutable.Map() - private var topicPidMap: immutable.Map[String, Seq[Int]] = immutable.Map() private def getConsumer(zkUtils: ZkUtils, bid: Int): Option[SimpleConsumer] = { try { @@ -79,7 +78,7 @@ object ConsumerOffsetChecker extends Logging { } } - private def processTopic(zkUtils: ZkUtils, group: String, topic: String) { + private def processTopic(topicPidMap: Map[String, Seq[Int]], zkUtils: ZkUtils, group: String, topic: String): Unit = { topicPidMap.get(topic) match { case Some(pids) => pids.sorted.foreach { @@ -154,7 +153,7 @@ object ConsumerOffsetChecker extends Logging { case None => zkUtils.getChildren(groupDirs.consumerGroupDir + "/owners").toList } - topicPidMap = immutable.Map(zkUtils.getPartitionsForTopics(topicList).toSeq:_*) + val topicPidMap = zkUtils.getPartitionsForTopics(topicList) val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq val channel = ClientUtils.channelToOffsetManager(group, zkUtils, channelSocketTimeoutMs, channelRetryBackoffMs) @@ -189,7 +188,7 @@ object ConsumerOffsetChecker extends Logging { println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner")) topicList.sorted.foreach { - topic => processTopic(zkUtils, group, topic) + topic => processTopic(topicPidMap, zkUtils, group, topic) } if (options.has("broker-info")) diff --git a/core/src/main/scala/kafka/utils/Pool.scala b/core/src/main/scala/kafka/utils/Pool.scala index df74f29aef93c..a1b5c2203a4ed 100644 --- a/core/src/main/scala/kafka/utils/Pool.scala +++ b/core/src/main/scala/kafka/utils/Pool.scala @@ -78,7 +78,8 @@ class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] { } def contains(id: K): Boolean = pool.containsKey(id) - + +// TODO: Return Option[V] def get(key: K): V = pool.get(key) def remove(key: K): V = pool.remove(key) diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 29e5d108edfc6..8b936dd9161b6 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -18,67 +18,62 @@ package kafka.utils import kafka.api.LeaderAndIsr -import kafka.common.TopicAndPartition import kafka.controller.{IsrChangeNotificationListener, LeaderIsrAndControllerEpoch} import kafka.utils.ZkUtils._ import org.apache.kafka.common.TopicPartition import org.apache.zookeeper.data.Stat -import scala.collection._ - object ReplicationUtils extends Logging { private val IsrChangeNotificationPrefix = "isr_change_" - def updateLeaderAndIsr(zkUtils: ZkUtils, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int, - zkVersion: Int): (Boolean,Int) = { +// TODO: Rewrite to return Option[LeaderAndIsr] + def updateLeaderAndIsr(zkUtils: ZkUtils, + topic: String, + partitionId: Int, + newLeaderAndIsr: LeaderAndIsr, + controllerEpoch: Int, + zkVersion: Int): (Boolean, Int) = { debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newLeaderAndIsr.isr.mkString(","))) val path = getTopicPartitionLeaderAndIsrPath(topic, partitionId) val newLeaderData = zkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) - // use the epoch of the controller that made the leadership decision, instead of the current controller epoch - val updatePersistentPath: (Boolean, Int) = zkUtils.conditionalUpdatePersistentPath(path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) - updatePersistentPath + zkUtils.conditionalUpdatePersistentPath(path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) } def propagateIsrChanges(zkUtils: ZkUtils, isrChangeSet: Set[TopicPartition]): Unit = { val isrChangeNotificationPath: String = zkUtils.createSequentialPersistentPath( ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix, - generateIsrChangeJson(isrChangeSet)) - debug("Added " + isrChangeNotificationPath + " for " + isrChangeSet) + generateIsrChangeJson(isrChangeSet) + ) + debug(s"Added $isrChangeNotificationPath for $isrChangeSet") } - def checkLeaderAndIsrZkData(zkUtils: ZkUtils, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { - try { - val writtenLeaderAndIsrInfo = zkUtils.readDataMaybeNull(path) - val writtenLeaderOpt = writtenLeaderAndIsrInfo._1 - val writtenStat = writtenLeaderAndIsrInfo._2 - val expectedLeader = parseLeaderAndIsr(expectedLeaderAndIsrInfo, path, writtenStat) - writtenLeaderOpt match { - case Some(writtenData) => - val writtenLeader = parseLeaderAndIsr(writtenData, path, writtenStat) - (expectedLeader,writtenLeader) match { - case (Some(expectedLeader),Some(writtenLeader)) => - if(expectedLeader == writtenLeader) - return (true, writtenStat.getVersion()) - case _ => - } - case None => - } - } catch { - case _: Exception => - } - (false,-1) +// TODO: Return might be rewritten to Option[Int], where None = (false, -1), Some(i) = (true, i = writtenStat.getVersion) + def checkLeaderAndIsrZkData(zkUtils: ZkUtils, path: String, expectedLeaderAndIsrInfo: String): (Boolean, Int) = { + val (writtenLeaderOpt, writtenStat) = zkUtils.readDataMaybeNull(path) + val expectedLeaderOpt = parseLeaderAndIsr(expectedLeaderAndIsrInfo, path, writtenStat) + + val ret = for { + writtenData <- writtenLeaderOpt + writtenLeader <- parseLeaderAndIsr(writtenData, path, writtenStat) + expectedLeader <- expectedLeaderOpt if expectedLeader == writtenLeader + } yield (true, writtenStat.getVersion) + + ret.getOrElse((false, -1)) } - def getLeaderIsrAndEpochForPartition(zkUtils: ZkUtils, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { + def getLeaderIsrAndEpochForPartition(zkUtils: ZkUtils, topic: String, partition: Int): Option[LeaderIsrAndControllerEpoch] = { val leaderAndIsrPath = getTopicPartitionLeaderAndIsrPath(topic, partition) val (leaderAndIsrOpt, stat) = zkUtils.readDataMaybeNull(leaderAndIsrPath) - leaderAndIsrOpt.flatMap(leaderAndIsrStr => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat)) + leaderAndIsrOpt.flatMap( + leaderAndIsrStr => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat) + ) } - private def parseLeaderAndIsr(leaderAndIsrStr: String, path: String, stat: Stat) - : Option[LeaderIsrAndControllerEpoch] = { - Json.parseFull(leaderAndIsrStr).flatMap {m => + private def parseLeaderAndIsr(leaderAndIsrStr: String, + path: String, + stat: Stat): Option[LeaderIsrAndControllerEpoch] = { + Json.parseFull(leaderAndIsrStr).map { m => val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] @@ -87,7 +82,8 @@ object ReplicationUtils extends Logging { val zkPathVersion = stat.getVersion debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for leaderAndIsrPath %s".format(leader, epoch, isr.toString(), zkPathVersion, path)) - Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch))} + LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch) + } } private def generateIsrChangeJson(isrChanges: Set[TopicPartition]): String = { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index f17d0bc5ce8b3..5009bb72cfe76 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -237,7 +237,7 @@ class ZkUtils(val zkClient: ZkClient, brokerIds.map(_.toInt).map(getBrokerInfo(_)).filter(_.isDefined).map(_.get) } - def getLeaderAndIsrForPartition(topic: String, partition: Int):Option[LeaderAndIsr] = { + def getLeaderAndIsrForPartition(topic: String, partition: Int): Option[LeaderAndIsr] = { ReplicationUtils.getLeaderIsrAndEpochForPartition(this, topic, partition).map(_.leaderAndIsr) } @@ -368,7 +368,7 @@ class ZkUtils(val zkClient: ZkClient, topicDirs.consumerOwnerDir + "/" + partition } - + // TODO: Why version is always 1, can we move this method to LeaderAndIsr? def leaderAndIsrZkData(leaderAndIsr: LeaderAndIsr, controllerEpoch: Int): String = { Json.encode(Map("version" -> 1, "leader" -> leaderAndIsr.leader, "leader_epoch" -> leaderAndIsr.leaderEpoch, "controller_epoch" -> controllerEpoch, "isr" -> leaderAndIsr.isr)) @@ -659,30 +659,27 @@ class ZkUtils(val zkClient: ZkClient, ret } - def getPartitionAssignmentForTopics(topics: Seq[String]): mutable.Map[String, collection.Map[Int, Seq[Int]]] = { - val ret = new mutable.HashMap[String, Map[Int, Seq[Int]]]() - topics.foreach{ topic => - val jsonPartitionMapOpt = readDataMaybeNull(getTopicPath(topic))._1 - val partitionMap = jsonPartitionMapOpt match { - case Some(jsonPartitionMap) => - Json.parseFull(jsonPartitionMap) match { - case Some(m) => m.asInstanceOf[Map[String, Any]].get("partitions") match { - case Some(replicaMap) => - val m1 = replicaMap.asInstanceOf[Map[String, Seq[Int]]] - m1.map(p => (p._1.toInt, p._2)) - case None => Map[Int, Seq[Int]]() - } - case None => Map[Int, Seq[Int]]() - } - case None => Map[Int, Seq[Int]]() + def getPartitionAssignmentForTopics(topics: Seq[String]): immutable.Map[String, immutable.Map[Int, Seq[Int]]] = { + val ret = topics.map { topic => + val partitionMapOpt = for { + jsonPartitionMap <- readDataMaybeNull(getTopicPath(topic))._1 + m <- Json.parseFull(jsonPartitionMap) + replicaMap <- m.asInstanceOf[immutable.Map[String, Any]].get("partitions") + } yield { + val m1 = replicaMap.asInstanceOf[immutable.Map[String, Seq[Int]]] + m1.map { case (partition, replicas) => partition.toInt -> replicas } } - debug("Partition map for /brokers/topics/%s is %s".format(topic, partitionMap)) - ret += (topic -> partitionMap) + + val partitionMap = partitionMapOpt.getOrElse(immutable.Map.empty[Int, Seq[Int]]) + debug(s"Partition map for /brokers/topics/$topic is $partitionMap") + + topic -> partitionMap } - ret + + ret.toMap } - def getPartitionsForTopics(topics: Seq[String]): mutable.Map[String, Seq[Int]] = { + def getPartitionsForTopics(topics: Seq[String]): immutable.Map[String, Seq[Int]] = { getPartitionAssignmentForTopics(topics).map { case (topic, partitionMap) => debug(s"partition assignment of /brokers/topics/$topic is $partitionMap") diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 7e3608009b8d1..899476be71718 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -78,8 +78,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) // make two partitions of the group topic to make sure some partitions are not owned by the coordinator - val ret = scala.collection.mutable.Map[String, scala.collection.Map[Int, Seq[Int]]]() - ret += (Topic.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) + val ret = Map(Topic.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala index 64b51c5b12cbb..72947b45d96b0 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala @@ -68,9 +68,7 @@ class GroupMetadataManagerTest { offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) // make two partitions of the group topic to make sure some partitions are not owned by the coordinator -// TODO: Get rid of mutable and collection - val ret = scala.collection.mutable.Map[String, scala.collection.Map[Int, Seq[Int]]]() - ret += (Topic.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) + val ret = Map(Topic.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(Topic.GroupMetadataTopicName))).andReturn(ret) diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index dbad66ffea70e..e70c1b54ebb1d 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -42,7 +42,11 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness { val topicDataMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, "versions" -> 2, "leader_epoch" -> 2,"isr" -> List(1,2))) - val topicDataLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(1,leaderEpoch,List(1,2),0), controllerEpoch) + val topicDataLeaderIsrAndControllerEpoch = + LeaderIsrAndControllerEpoch( + LeaderAndIsr(1, leaderEpoch, List(1, 2), 0), + controllerEpoch + ) @Before override def setUp() { @@ -86,14 +90,14 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness { "my-topic-test", partitionId, newLeaderAndIsr2, controllerEpoch, zkVersion + 1) assertTrue(updateSucceeded2) // returns true with existing zkVersion - assertEquals(newZkVersion2,1) + assertEquals(newZkVersion2, 1) // mismatched zkVersion and leaderEpoch val newLeaderAndIsr3 = new LeaderAndIsr(brokerId, leaderEpoch + 1, replicas, zkVersion + 1) val (updateSucceeded3,newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkUtils, "my-topic-test", partitionId, newLeaderAndIsr3, controllerEpoch, zkVersion + 1) assertFalse(updateSucceeded3) - assertEquals(newZkVersion3,-1) + assertEquals(newZkVersion3, -1) } @Test diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 1e7385b2bd8a6..313eab5f94b4a 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -681,31 +681,25 @@ object TestUtils extends Logging { new ProducerRequest(correlationId, clientId, acks.toShort, timeout, collection.mutable.Map(data:_*)) } - def makeLeaderForPartition(zkUtils: ZkUtils, topic: String, + def makeLeaderForPartition(zkUtils: ZkUtils, + topic: String, leaderPerPartitionMap: scala.collection.immutable.Map[Int, Int], - controllerEpoch: Int) { - leaderPerPartitionMap.foreach - { - leaderForPartition => { - val partition = leaderForPartition._1 - val leader = leaderForPartition._2 + controllerEpoch: Int): Unit = { + leaderPerPartitionMap.foreach { + case (partition, leader) => try{ - val currentLeaderAndIsrOpt = zkUtils.getLeaderAndIsrForPartition(topic, partition) - var newLeaderAndIsr: LeaderAndIsr = null - if(currentLeaderAndIsrOpt.isEmpty) - newLeaderAndIsr = LeaderAndIsr(leader, List(leader)) - else{ - newLeaderAndIsr = currentLeaderAndIsrOpt.get - newLeaderAndIsr.leader = leader - newLeaderAndIsr.leaderEpoch += 1 - newLeaderAndIsr.zkVersion += 1 + val newLeaderAndIsr = zkUtils.getLeaderAndIsrForPartition(topic, partition) match { + case Some(current) => current.newLeader(leader) + case None => LeaderAndIsr(leader, List(leader)) } - zkUtils.updatePersistentPath(getTopicPartitionLeaderAndIsrPath(topic, partition), - zkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch)) + + zkUtils.updatePersistentPath( + getTopicPartitionLeaderAndIsrPath(topic, partition), + zkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) + ) } catch { - case oe: Throwable => error("Error while electing leader for partition [%s,%d]".format(topic, partition), oe) + case oe: Throwable => error(s"Error while electing leader for partition [$topic, $partition]", oe) } - } } }