|
|
@ -405,18 +405,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, |
|
|
|
val brokers = getAllBrokersInCluster(zkClient) |
|
|
|
val brokers = getAllBrokersInCluster(zkClient) |
|
|
|
val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, brokers, config.clientId).topicsMetadata |
|
|
|
val topicsMetadata = ClientUtils.fetchTopicMetadata(myTopicThreadIdsMap.keySet, brokers, config.clientId).topicsMetadata |
|
|
|
val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]] |
|
|
|
val partitionsPerTopicMap = new mutable.HashMap[String, Seq[Int]] |
|
|
|
val leaderIdForPartitionsMap = new mutable.HashMap[(String, Int), Int] |
|
|
|
|
|
|
|
topicsMetadata.foreach(m =>{ |
|
|
|
topicsMetadata.foreach(m =>{ |
|
|
|
val topic = m.topic |
|
|
|
val topic = m.topic |
|
|
|
val partitions = m.partitionsMetadata.map(m1 => m1.partitionId) |
|
|
|
val partitions = m.partitionsMetadata.map(m1 => m1.partitionId) |
|
|
|
partitionsPerTopicMap.put(topic, partitions) |
|
|
|
partitionsPerTopicMap.put(topic, partitions) |
|
|
|
m.partitionsMetadata.foreach(pmd =>{ |
|
|
|
|
|
|
|
val partitionId = pmd.partitionId |
|
|
|
|
|
|
|
val leaderOpt = pmd.leader |
|
|
|
|
|
|
|
if(leaderOpt.isDefined) |
|
|
|
|
|
|
|
leaderIdForPartitionsMap.put((topic, partitionId), leaderOpt.get.id) |
|
|
|
|
|
|
|
}) |
|
|
|
|
|
|
|
}) |
|
|
|
}) |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* fetchers must be stopped to avoid data duplication, since if the current |
|
|
|
* fetchers must be stopped to avoid data duplication, since if the current |
|
|
|
* rebalancing attempt fails, the partitions that are released could be owned by another consumer. |
|
|
|
* rebalancing attempt fails, the partitions that are released could be owned by another consumer. |
|
|
@ -459,7 +453,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, |
|
|
|
for (i <- startPart until startPart + nParts) { |
|
|
|
for (i <- startPart until startPart + nParts) { |
|
|
|
val partition = curPartitions(i) |
|
|
|
val partition = curPartitions(i) |
|
|
|
info(consumerThreadId + " attempting to claim partition " + partition) |
|
|
|
info(consumerThreadId + " attempting to claim partition " + partition) |
|
|
|
addPartitionTopicInfo(currentTopicRegistry, leaderIdForPartitionsMap, topicDirs, partition, topic, consumerThreadId) |
|
|
|
addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) |
|
|
|
// record the partition ownership decision |
|
|
|
// record the partition ownership decision |
|
|
|
partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) |
|
|
|
partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) |
|
|
|
} |
|
|
|
} |
|
|
@ -576,39 +570,22 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]], |
|
|
|
private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]], |
|
|
|
leaderIdForPartitionsMap: Map[(String, Int), Int], |
|
|
|
|
|
|
|
topicDirs: ZKGroupTopicDirs, partition: Int, |
|
|
|
topicDirs: ZKGroupTopicDirs, partition: Int, |
|
|
|
topic: String, consumerThreadId: String) { |
|
|
|
topic: String, consumerThreadId: String) { |
|
|
|
val partTopicInfoMap = currentTopicRegistry.get(topic) |
|
|
|
val partTopicInfoMap = currentTopicRegistry.get(topic) |
|
|
|
|
|
|
|
|
|
|
|
// find the leader for this partition |
|
|
|
|
|
|
|
val leaderOpt = leaderIdForPartitionsMap.get((topic, partition)) |
|
|
|
|
|
|
|
leaderOpt match { |
|
|
|
|
|
|
|
case None => throw new NoBrokersForPartitionException("No leader available for partition %d on topic %s". |
|
|
|
|
|
|
|
format(partition, topic)) |
|
|
|
|
|
|
|
case Some(l) => debug("Leader for partition %d for topic %s is %d".format(partition, topic, l)) |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
val leader = leaderOpt.get |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
val znode = topicDirs.consumerOffsetDir + "/" + partition |
|
|
|
val znode = topicDirs.consumerOffsetDir + "/" + partition |
|
|
|
val offsetString = readDataMaybeNull(zkClient, znode)._1 |
|
|
|
val offsetString = readDataMaybeNull(zkClient, znode)._1 |
|
|
|
// If first time starting a consumer, set the initial offset based on the config |
|
|
|
// If first time starting a consumer, set the initial offset to -1 |
|
|
|
val offset = |
|
|
|
val offset = |
|
|
|
offsetString match { |
|
|
|
offsetString match { |
|
|
|
case Some(offsetStr) => offsetStr.toLong |
|
|
|
case Some(offsetStr) => offsetStr.toLong |
|
|
|
case None => |
|
|
|
case None => PartitionTopicInfo.InvalidOffset |
|
|
|
config.autoOffsetReset match { |
|
|
|
|
|
|
|
case OffsetRequest.SmallestTimeString => |
|
|
|
|
|
|
|
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.EarliestTime, config.clientId) |
|
|
|
|
|
|
|
case OffsetRequest.LargestTimeString => |
|
|
|
|
|
|
|
SimpleConsumer.earliestOrLatestOffset(zkClient, topic, leader, partition, OffsetRequest.LatestTime, config.clientId) |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) |
|
|
|
val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) |
|
|
|
val consumedOffset = new AtomicLong(offset) |
|
|
|
val consumedOffset = new AtomicLong(offset) |
|
|
|
val fetchedOffset = new AtomicLong(offset) |
|
|
|
val fetchedOffset = new AtomicLong(offset) |
|
|
|
val partTopicInfo = new PartitionTopicInfo(topic, |
|
|
|
val partTopicInfo = new PartitionTopicInfo(topic, |
|
|
|
leader, |
|
|
|
|
|
|
|
partition, |
|
|
|
partition, |
|
|
|
queue, |
|
|
|
queue, |
|
|
|
consumedOffset, |
|
|
|
consumedOffset, |
|
|
|