mirror of https://github.com/apache/kafka.git
revisit the become leader and become follower state change operations using V3 design; patched by Yang Ye; reviewed by Neha Narkhede and Jun Rao; kafka-343
git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8@1367619 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
19fbdb3099
commit
2a9d42c243
|
@ -91,10 +91,10 @@ object AdminUtils extends Logging {
|
|||
topics.map { topic =>
|
||||
if (ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) {
|
||||
val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic).iterator).get(topic).get
|
||||
val sortedPartitions = topicPartitionAssignment.toList.sortWith( (m1,m2) => m1._1.toInt < m2._1.toInt )
|
||||
val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1)
|
||||
|
||||
val partitionMetadata = sortedPartitions.map { partitionMap =>
|
||||
val partition = partitionMap._1.toInt
|
||||
val partition = partitionMap._1
|
||||
val replicas = partitionMap._2
|
||||
val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partition)
|
||||
val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
|
||||
|
|
|
@ -25,39 +25,54 @@ import collection.mutable.HashMap
|
|||
|
||||
|
||||
object LeaderAndISR {
|
||||
val initialLeaderEpoch: Int = 0
|
||||
val initialZKVersion: Int = 0
|
||||
def readFrom(buffer: ByteBuffer): LeaderAndISR = {
|
||||
val leader = buffer.getInt
|
||||
val leaderGenId = buffer.getInt
|
||||
val ISRString = Utils.readShortString(buffer, "UTF-8")
|
||||
val ISR = ISRString.split(",").map(_.toInt).toList
|
||||
val zkVersion = buffer.getLong
|
||||
val zkVersion = buffer.getInt
|
||||
new LeaderAndISR(leader, leaderGenId, ISR, zkVersion)
|
||||
}
|
||||
}
|
||||
|
||||
case class LeaderAndISR(leader: Int, leaderEpoc: Int, ISR: List[Int], zkVersion: Long){
|
||||
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)
|
||||
|
||||
def writeTo(buffer: ByteBuffer) {
|
||||
buffer.putInt(leader)
|
||||
buffer.putInt(leaderEpoc)
|
||||
buffer.putInt(leaderEpoch)
|
||||
Utils.writeShortString(buffer, ISR.mkString(","), "UTF-8")
|
||||
buffer.putLong(zkVersion)
|
||||
buffer.putInt(zkVersion)
|
||||
}
|
||||
|
||||
def sizeInBytes(): Int = {
|
||||
val size = 4 + 4 + (2 + ISR.mkString(",").length) + 8
|
||||
val size = 4 + 4 + (2 + ISR.mkString(",").length) + 4
|
||||
size
|
||||
}
|
||||
|
||||
override def toString(): String = {
|
||||
val jsonDataMap = new HashMap[String, String]
|
||||
jsonDataMap.put("leader", leader.toString)
|
||||
jsonDataMap.put("leaderEpoch", leaderEpoch.toString)
|
||||
jsonDataMap.put("ISR", ISR.mkString(","))
|
||||
Utils.stringMapToJsonString(jsonDataMap)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
object LeaderAndISRRequest {
|
||||
val CurrentVersion = 1.shortValue()
|
||||
val DefaultClientId = ""
|
||||
val IsInit: Boolean = true
|
||||
val NotInit: Boolean = false
|
||||
val DefaultAckTimeout: Int = 1000
|
||||
|
||||
def readFrom(buffer: ByteBuffer): LeaderAndISRRequest = {
|
||||
val versionId = buffer.getShort
|
||||
val clientId = Utils.readShortString(buffer)
|
||||
val isInit = buffer.get()
|
||||
val isInit = if(buffer.get() == 1.toByte) true else false
|
||||
val ackTimeoutMs = buffer.getInt
|
||||
val leaderAndISRRequestCount = buffer.getInt
|
||||
val leaderAndISRInfos = new HashMap[(String, Int), LeaderAndISR]
|
||||
|
@ -76,19 +91,18 @@ object LeaderAndISRRequest {
|
|||
|
||||
case class LeaderAndISRRequest (versionId: Short,
|
||||
clientId: String,
|
||||
isInit: Byte,
|
||||
isInit: Boolean,
|
||||
ackTimeoutMs: Int,
|
||||
leaderAndISRInfos:
|
||||
Map[(String, Int), LeaderAndISR])
|
||||
leaderAndISRInfos: Map[(String, Int), LeaderAndISR])
|
||||
extends RequestOrResponse(Some(RequestKeys.LeaderAndISRRequest)) {
|
||||
def this(isInit: Byte, ackTimeoutMs: Int, leaderAndISRInfos: Map[(String, Int), LeaderAndISR]) = {
|
||||
this(LeaderAndISRRequest.CurrentVersion, LeaderAndISRRequest.DefaultClientId, isInit, ackTimeoutMs, leaderAndISRInfos)
|
||||
def this(isInit: Boolean, leaderAndISRInfos: Map[(String, Int), LeaderAndISR]) = {
|
||||
this(LeaderAndISRRequest.CurrentVersion, LeaderAndISRRequest.DefaultClientId, isInit, LeaderAndISRRequest.DefaultAckTimeout, leaderAndISRInfos)
|
||||
}
|
||||
|
||||
def writeTo(buffer: ByteBuffer) {
|
||||
buffer.putShort(versionId)
|
||||
Utils.writeShortString(buffer, clientId)
|
||||
buffer.put(isInit)
|
||||
buffer.put(if(isInit) 1.toByte else 0.toByte)
|
||||
buffer.putInt(ackTimeoutMs)
|
||||
buffer.putInt(leaderAndISRInfos.size)
|
||||
for((key, value) <- leaderAndISRInfos){
|
||||
|
|
|
@ -26,6 +26,7 @@ import collection.mutable.Set
|
|||
object StopReplicaRequest {
|
||||
val CurrentVersion = 1.shortValue()
|
||||
val DefaultClientId = ""
|
||||
val DefaultAckTimeout = 100
|
||||
|
||||
def readFrom(buffer: ByteBuffer): StopReplicaRequest = {
|
||||
val versionId = buffer.getShort
|
||||
|
@ -43,10 +44,10 @@ object StopReplicaRequest {
|
|||
case class StopReplicaRequest(versionId: Short,
|
||||
clientId: String,
|
||||
ackTimeoutMs: Int,
|
||||
stopReplicaSet: Set[(String, Int)]
|
||||
) extends RequestOrResponse(Some(RequestKeys.StopReplicaRequest)) {
|
||||
def this(ackTimeoutMs: Int, stopReplicaSet: Set[(String, Int)]) = {
|
||||
this(StopReplicaRequest.CurrentVersion, StopReplicaRequest.DefaultClientId, ackTimeoutMs, stopReplicaSet)
|
||||
stopReplicaSet: Set[(String, Int)])
|
||||
extends RequestOrResponse(Some(RequestKeys.StopReplicaRequest)) {
|
||||
def this(stopReplicaSet: Set[(String, Int)]) = {
|
||||
this(StopReplicaRequest.CurrentVersion, StopReplicaRequest.DefaultClientId, StopReplicaRequest.DefaultAckTimeout, stopReplicaSet)
|
||||
}
|
||||
|
||||
def writeTo(buffer: ByteBuffer) {
|
||||
|
@ -67,4 +68,4 @@ case class StopReplicaRequest(versionId: Short,
|
|||
}
|
||||
size
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package kafka.cluster
|
|||
|
||||
import kafka.utils.Utils._
|
||||
import java.nio.ByteBuffer
|
||||
import kafka.common.KafkaException
|
||||
import kafka.common.BrokerNotExistException
|
||||
|
||||
/**
|
||||
* A Kafka broker
|
||||
|
@ -28,7 +28,7 @@ private[kafka] object Broker {
|
|||
|
||||
def createBroker(id: Int, brokerInfoString: String): Broker = {
|
||||
if(brokerInfoString == null)
|
||||
throw new KafkaException("Broker id %s does not exist".format(id))
|
||||
throw new BrokerNotExistException("Broker id %s does not exist".format(id))
|
||||
val brokerInfo = brokerInfoString.split(":")
|
||||
new Broker(id, brokerInfo(0), brokerInfo(1), brokerInfo(2).toInt)
|
||||
}
|
||||
|
|
|
@ -16,11 +16,12 @@
|
|||
*/
|
||||
package kafka.cluster
|
||||
|
||||
import kafka.utils.{SystemTime, Time, Logging}
|
||||
import org.I0Itec.zkclient.ZkClient
|
||||
import kafka.utils.ZkUtils._
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import scala.collection._
|
||||
import kafka.utils.{ZkUtils, SystemTime, Time}
|
||||
import kafka.common.{KafkaException, LeaderNotAvailableException}
|
||||
import org.I0Itec.zkclient.ZkClient
|
||||
import kafka.utils.Logging
|
||||
|
||||
/**
|
||||
* Data structure that represents a topic partition. The leader maintains the AR, ISR, CUR, RAR
|
||||
|
@ -119,14 +120,8 @@ class Partition(val topic: String,
|
|||
}
|
||||
|
||||
def updateISR(newISR: Set[Int], zkClientOpt: Option[ZkClient] = None) {
|
||||
try {
|
||||
try{
|
||||
leaderISRUpdateLock.lock()
|
||||
zkClientOpt match {
|
||||
case Some(zkClient) =>
|
||||
// update ISR in ZK
|
||||
updateISRInZk(newISR, zkClient)
|
||||
case None =>
|
||||
}
|
||||
// update partition's ISR in cache
|
||||
inSyncReplicas = newISR.map {r =>
|
||||
getReplica(r) match {
|
||||
|
@ -135,29 +130,22 @@ class Partition(val topic: String,
|
|||
}
|
||||
}
|
||||
info("Updated ISR for topic %s partition %d to %s in cache".format(topic, partitionId, newISR.mkString(",")))
|
||||
}catch {
|
||||
case e => throw new KafkaException("Failed to update ISR for topic %s ".format(topic) +
|
||||
"partition %d to %s".format(partitionId, newISR.mkString(",")), e)
|
||||
}finally {
|
||||
if(zkClientOpt.isDefined){
|
||||
val zkClient = zkClientOpt.get
|
||||
val curLeaderAndISR = ZkUtils.getLeaderAndISRForPartition(zkClient, topic, partitionId)
|
||||
curLeaderAndISR match {
|
||||
case None =>
|
||||
throw new IllegalStateException("The leaderAndISR info for partition [%s, %s] is not in Zookeeper".format(topic, partitionId))
|
||||
case Some(m) =>
|
||||
m.ISR = newISR.toList
|
||||
ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndISRPath(topic, partitionId), m.toString)
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
leaderISRUpdateLock.unlock()
|
||||
}
|
||||
}
|
||||
|
||||
private def updateISRInZk(newISR: Set[Int], zkClient: ZkClient) = {
|
||||
val replicaListAndEpochString = readDataMaybeNull(zkClient, getTopicPartitionInSyncPath(topic, partitionId.toString))
|
||||
if(replicaListAndEpochString == null) {
|
||||
throw new LeaderNotAvailableException(("Illegal partition state. ISR cannot be updated for topic " +
|
||||
"%s partition %d since leader and ISR does not exist in ZK".format(topic, partitionId)))
|
||||
}
|
||||
else {
|
||||
val replicasAndEpochInfo = replicaListAndEpochString.split(";")
|
||||
val epoch = replicasAndEpochInfo.last
|
||||
updatePersistentPath(zkClient, getTopicPartitionInSyncPath(topic, partitionId.toString),
|
||||
"%s;%s".format(newISR.mkString(","), epoch))
|
||||
info("Updated ISR for topic %s partition %d to %s in ZK".format(topic, partitionId, newISR.mkString(",")))
|
||||
}
|
||||
}
|
||||
|
||||
override def equals(that: Any): Boolean = {
|
||||
if(!(that.isInstanceOf[Partition]))
|
||||
return false
|
||||
|
@ -180,4 +168,4 @@ class Partition(val topic: String,
|
|||
partitionString.append("; In Sync replicas: " + inSyncReplicas.map(_.brokerId).mkString(","))
|
||||
partitionString.toString()
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.common
|
||||
|
||||
class BrokerNotExistException(message: String) extends RuntimeException(message) {
|
||||
def this() = this(null)
|
||||
}
|
|
@ -39,7 +39,8 @@ object ErrorMapping {
|
|||
val NotLeaderForPartitionCode : Short = 7
|
||||
val UnknownTopicCode : Short = 8
|
||||
val RequestTimedOutCode: Short = 9
|
||||
val ReplicaNotAvailableCode: Short = 10
|
||||
val BrokerNotExistInZookeeperCode: Short = 10
|
||||
val ReplicaNotAvailableCode: Short = 11
|
||||
|
||||
private val exceptionToCode =
|
||||
Map[Class[Throwable], Short](
|
||||
|
@ -51,8 +52,9 @@ object ErrorMapping {
|
|||
classOf[NotLeaderForPartitionException].asInstanceOf[Class[Throwable]] -> NotLeaderForPartitionCode,
|
||||
classOf[LeaderNotAvailableException].asInstanceOf[Class[Throwable]] -> LeaderNotAvailableCode,
|
||||
classOf[RequestTimedOutException].asInstanceOf[Class[Throwable]] -> RequestTimedOutCode,
|
||||
classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode,
|
||||
classOf[UnknownTopicException].asInstanceOf[Class[Throwable]] -> UnknownTopicCode
|
||||
classOf[UnknownTopicException].asInstanceOf[Class[Throwable]] -> UnknownTopicCode,
|
||||
classOf[BrokerNotExistException].asInstanceOf[Class[Throwable]] -> BrokerNotExistInZookeeperCode,
|
||||
classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode
|
||||
).withDefaultValue(UnknownCode)
|
||||
|
||||
/* invert the mapping */
|
||||
|
|
|
@ -88,8 +88,7 @@ class ConsumerFetcherManager(private val consumerIdString: String,
|
|||
|
||||
|
||||
override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = {
|
||||
new ConsumerFetcherThread("ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id),
|
||||
config, sourceBroker, this)
|
||||
new ConsumerFetcherThread("ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), config, sourceBroker, this)
|
||||
}
|
||||
|
||||
def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) {
|
||||
|
|
|
@ -66,7 +66,7 @@ private[kafka] object TopicCount extends Logging {
|
|||
consumerId: String,
|
||||
zkClient: ZkClient) : TopicCount = {
|
||||
val dirs = new ZKGroupDirs(group)
|
||||
val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)
|
||||
val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1
|
||||
val hasWhitelist = topicCountString.startsWith(WHITELIST_MARKER)
|
||||
val hasBlacklist = topicCountString.startsWith(BLACKLIST_MARKER)
|
||||
|
||||
|
|
|
@ -296,7 +296,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
try {
|
||||
val topicDirs = new ZKGroupTopicDirs(config.groupId, topic)
|
||||
val znode = topicDirs.consumerOffsetDir + "/" + partitionId
|
||||
val offsetString = readDataMaybeNull(zkClient, znode)
|
||||
val offsetString = readDataMaybeNull(zkClient, znode)._1
|
||||
if (offsetString != null)
|
||||
return offsetString.toLong
|
||||
else
|
||||
|
@ -416,7 +416,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
}
|
||||
}
|
||||
|
||||
private def deletePartitionOwnershipFromZK(topic: String, partition: String) {
|
||||
private def deletePartitionOwnershipFromZK(topic: String, partition: Int) {
|
||||
val topicDirs = new ZKGroupTopicDirs(group, topic)
|
||||
val znode = topicDirs.consumerOwnerDir + "/" + partition
|
||||
deletePath(zkClient, znode)
|
||||
|
@ -427,7 +427,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
info("Releasing partition ownership")
|
||||
for ((topic, infos) <- localTopicRegistry) {
|
||||
for(partition <- infos.keys)
|
||||
deletePartitionOwnershipFromZK(topic, partition.toString)
|
||||
deletePartitionOwnershipFromZK(topic, partition)
|
||||
localTopicRegistry.remove(topic)
|
||||
}
|
||||
}
|
||||
|
@ -484,7 +484,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
|
||||
releasePartitionOwnership(topicRegistry)
|
||||
|
||||
var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]()
|
||||
var partitionOwnershipDecision = new collection.mutable.HashMap[(String, Int), String]()
|
||||
var currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]
|
||||
|
||||
for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) {
|
||||
|
@ -492,7 +492,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
|
||||
val topicDirs = new ZKGroupTopicDirs(group, topic)
|
||||
val curConsumers = consumersPerTopicMap.get(topic).get
|
||||
var curPartitions: Seq[String] = partitionsPerTopicMap.get(topic).get
|
||||
var curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get
|
||||
|
||||
val nPartsPerConsumer = curPartitions.size / curConsumers.size
|
||||
val nConsumersWithExtraPart = curPartitions.size % curConsumers.size
|
||||
|
@ -602,13 +602,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
}
|
||||
}
|
||||
|
||||
private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = {
|
||||
var successfullyOwnedPartitions : List[(String, String)] = Nil
|
||||
private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, Int), String]): Boolean = {
|
||||
var successfullyOwnedPartitions : List[(String, Int)] = Nil
|
||||
val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner =>
|
||||
val topic = partitionOwner._1._1
|
||||
val partition = partitionOwner._1._2
|
||||
val consumerThreadId = partitionOwner._2
|
||||
val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic,partition)
|
||||
val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition)
|
||||
try {
|
||||
createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId)
|
||||
info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic)
|
||||
|
@ -633,29 +633,29 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
}
|
||||
|
||||
private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]],
|
||||
topicDirs: ZKGroupTopicDirs, partition: String,
|
||||
topicDirs: ZKGroupTopicDirs, partition: Int,
|
||||
topic: String, consumerThreadId: String) {
|
||||
val partTopicInfoMap = currentTopicRegistry.get(topic)
|
||||
|
||||
// find the leader for this partition
|
||||
val leaderOpt = getLeaderForPartition(zkClient, topic, partition.toInt)
|
||||
val leaderOpt = getLeaderForPartition(zkClient, topic, partition)
|
||||
leaderOpt match {
|
||||
case None => throw new NoBrokersForPartitionException("No leader available for partition %s on topic %s".
|
||||
case None => throw new NoBrokersForPartitionException("No leader available for partition %d on topic %s".
|
||||
format(partition, topic))
|
||||
case Some(l) => debug("Leader for partition %s for topic %s is %d".format(partition, topic, l))
|
||||
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 offsetString = readDataMaybeNull(zkClient, znode)
|
||||
val offsetString = readDataMaybeNull(zkClient, znode)._1
|
||||
// If first time starting a consumer, set the initial offset based on the config
|
||||
var offset : Long = 0L
|
||||
if (offsetString == null)
|
||||
offset = config.autoOffsetReset match {
|
||||
case OffsetRequest.SmallestTimeString =>
|
||||
earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.EarliestTime)
|
||||
earliestOrLatestOffset(topic, leader, partition, OffsetRequest.EarliestTime)
|
||||
case OffsetRequest.LargestTimeString =>
|
||||
earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.LatestTime)
|
||||
earliestOrLatestOffset(topic, leader, partition, OffsetRequest.LatestTime)
|
||||
case _ =>
|
||||
throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig")
|
||||
}
|
||||
|
@ -666,12 +666,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
val fetchedOffset = new AtomicLong(offset)
|
||||
val partTopicInfo = new PartitionTopicInfo(topic,
|
||||
leader,
|
||||
partition.toInt,
|
||||
partition,
|
||||
queue,
|
||||
consumedOffset,
|
||||
fetchedOffset,
|
||||
new AtomicInteger(config.fetchSize))
|
||||
partTopicInfoMap.put(partition.toInt, partTopicInfo)
|
||||
partTopicInfoMap.put(partition, partTopicInfo)
|
||||
debug(partTopicInfo + " selected new offset " + offset)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -29,7 +29,6 @@ import kafka.common.{KafkaException, InvalidMessageSizeException, OffsetOutOfRan
|
|||
|
||||
object Log {
|
||||
val FileSuffix = ".kafka"
|
||||
val hwFileName = "highwatermark"
|
||||
|
||||
/**
|
||||
* Find a given range object in a list of ranges by a value in that range. Does a binary search over the ranges
|
||||
|
@ -115,8 +114,8 @@ class LogSegment(val file: File, val messageSet: FileMessageSet, val start: Long
|
|||
* An append-only log for storing messages.
|
||||
*/
|
||||
@threadsafe
|
||||
private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, val needRecovery: Boolean)
|
||||
extends Logging {
|
||||
private[kafka] class Log( val dir: File, val maxSize: Long, val flushInterval: Int, val needRecovery: Boolean, brokerId: Int = 0) extends Logging {
|
||||
this.logIdent = "Kafka Log on Broker " + brokerId + ", "
|
||||
|
||||
import kafka.log.Log._
|
||||
|
||||
|
@ -126,7 +125,7 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
/* The current number of unflushed messages appended to the write */
|
||||
private val unflushed = new AtomicInteger(0)
|
||||
|
||||
/* last time it was flushed */
|
||||
/* last time it was flushed */
|
||||
private val lastflushedTime = new AtomicLong(System.currentTimeMillis)
|
||||
|
||||
/* The actual segments of the log */
|
||||
|
@ -191,8 +190,7 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
val curr = segments.get(i)
|
||||
val next = segments.get(i+1)
|
||||
if(curr.start + curr.size != next.start)
|
||||
throw new KafkaException("The following segments don't validate: " +
|
||||
curr.file.getAbsolutePath() + ", " + next.file.getAbsolutePath())
|
||||
throw new KafkaException("The following segments don't validate: " + curr.file.getAbsolutePath() + ", " + next.file.getAbsolutePath())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -231,13 +229,12 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
BrokerTopicStat.getBrokerTopicStat(topicName).recordMessagesIn(numberOfMessages)
|
||||
BrokerTopicStat.getBrokerAllTopicStat.recordMessagesIn(numberOfMessages)
|
||||
logStats.recordAppendedMessages(numberOfMessages)
|
||||
|
||||
|
||||
// truncate the message set's buffer upto validbytes, before appending it to the on-disk log
|
||||
val validByteBuffer = messages.getBuffer.duplicate()
|
||||
val messageSetValidBytes = messages.validBytes
|
||||
if(messageSetValidBytes > Int.MaxValue || messageSetValidBytes < 0)
|
||||
throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes +
|
||||
" Message set cannot be appended to log. Possible causes are corrupted produce requests")
|
||||
throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
|
||||
|
||||
validByteBuffer.limit(messageSetValidBytes.asInstanceOf[Int])
|
||||
val validMessages = new ByteBufferMessageSet(validByteBuffer)
|
||||
|
@ -347,12 +344,11 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
if (unflushed.get == 0) return
|
||||
|
||||
lock synchronized {
|
||||
debug("Flushing log '" + name + "' last flushed: " + getLastFlushedTime + " current time: " +
|
||||
System.currentTimeMillis)
|
||||
debug("Flushing log '" + name + "' last flushed: " + getLastFlushedTime + " current time: " + System.currentTimeMillis)
|
||||
segments.view.last.messageSet.flush()
|
||||
unflushed.set(0)
|
||||
lastflushedTime.set(System.currentTimeMillis)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getOffsetsBefore(request: OffsetRequest): Array[Long] = {
|
||||
|
@ -375,15 +371,15 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
case OffsetRequest.EarliestTime =>
|
||||
startIndex = 0
|
||||
case _ =>
|
||||
var isFound = false
|
||||
debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2)))
|
||||
startIndex = offsetTimeArray.length - 1
|
||||
while (startIndex >= 0 && !isFound) {
|
||||
if (offsetTimeArray(startIndex)._2 <= request.time)
|
||||
isFound = true
|
||||
else
|
||||
startIndex -=1
|
||||
}
|
||||
var isFound = false
|
||||
debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2)))
|
||||
startIndex = offsetTimeArray.length - 1
|
||||
while (startIndex >= 0 && !isFound) {
|
||||
if (offsetTimeArray(startIndex)._2 <= request.time)
|
||||
isFound = true
|
||||
else
|
||||
startIndex -=1
|
||||
}
|
||||
}
|
||||
|
||||
val retSize = request.maxNumOffsets.min(startIndex + 1)
|
||||
|
@ -408,7 +404,13 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
}
|
||||
}
|
||||
|
||||
/* Attemps to delete all provided segments from a log and returns how many it was able to */
|
||||
|
||||
def deleteWholeLog():Unit = {
|
||||
deleteSegments(segments.contents.get())
|
||||
Utils.rm(dir)
|
||||
}
|
||||
|
||||
/* Attempts to delete all provided segments from a log and returns how many it was able to */
|
||||
def deleteSegments(segments: Seq[LogSegment]): Int = {
|
||||
var total = 0
|
||||
for(segment <- segments) {
|
||||
|
@ -424,30 +426,27 @@ private[kafka] class Log(val dir: File, val maxSize: Long, val flushInterval: In
|
|||
}
|
||||
|
||||
def truncateTo(targetOffset: Long) {
|
||||
// find the log segment that has this hw
|
||||
val segmentToBeTruncated = segments.view.find(segment =>
|
||||
targetOffset >= segment.start && targetOffset < segment.absoluteEndOffset)
|
||||
// find the log segment that has this hw
|
||||
val segmentToBeTruncated = segments.view.find(
|
||||
segment => targetOffset >= segment.start && targetOffset < segment.absoluteEndOffset)
|
||||
|
||||
segmentToBeTruncated match {
|
||||
case Some(segment) =>
|
||||
val truncatedSegmentIndex = segments.view.indexOf(segment)
|
||||
segments.truncLast(truncatedSegmentIndex)
|
||||
segment.truncateTo(targetOffset)
|
||||
info("Truncated log segment %s to highwatermark %d".format(segment.file.getAbsolutePath, targetOffset))
|
||||
case None =>
|
||||
assert(targetOffset <= segments.view.last.absoluteEndOffset,
|
||||
"Last checkpointed hw %d cannot be greater than the latest message offset %d in the log %s".
|
||||
format(targetOffset, segments.view.last.absoluteEndOffset, segments.view.last.file.getAbsolutePath))
|
||||
error("Cannot truncate log to %d since the log start offset is %d and end offset is %d"
|
||||
.format(targetOffset, segments.view.head.start, segments.view.last.absoluteEndOffset))
|
||||
}
|
||||
segmentToBeTruncated match {
|
||||
case Some(segment) =>
|
||||
val truncatedSegmentIndex = segments.view.indexOf(segment)
|
||||
segments.truncLast(truncatedSegmentIndex)
|
||||
segment.truncateTo(targetOffset)
|
||||
info("Truncated log segment %s to highwatermark %d".format(segment.file.getAbsolutePath, targetOffset))
|
||||
case None =>
|
||||
assert(targetOffset <= segments.view.last.absoluteEndOffset, "Last checkpointed hw %d cannot be greater than the latest message offset %d in the log %s".format(targetOffset, segments.view.last.absoluteEndOffset, segments.view.last.file.getAbsolutePath))
|
||||
error("Cannot truncate log to %d since the log start offset is %d and end offset is %d".format(targetOffset, segments.view.head.start, segments.view.last.absoluteEndOffset))
|
||||
}
|
||||
|
||||
val segmentsToBeDeleted = segments.view.filter(segment => segment.start > targetOffset)
|
||||
if(segmentsToBeDeleted.size < segments.view.size) {
|
||||
val segmentsToBeDeleted = segments.view.filter(segment => segment.start > targetOffset)
|
||||
if(segmentsToBeDeleted.size < segments.view.size) {
|
||||
val numSegmentsDeleted = deleteSegments(segmentsToBeDeleted)
|
||||
if(numSegmentsDeleted != segmentsToBeDeleted.size)
|
||||
error("Failed to delete some segments during log recovery")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def topicName():String = {
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -35,7 +35,7 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
val logCleanupIntervalMs: Long,
|
||||
val logCleanupDefaultAgeMs: Long,
|
||||
needRecovery: Boolean) extends Logging {
|
||||
|
||||
|
||||
val logDir: File = new File(config.logDir)
|
||||
private val numPartitions = config.numPartitions
|
||||
private val maxSize: Long = config.logFileSize
|
||||
|
@ -44,6 +44,7 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
private val logFlushIntervals = config.flushIntervalMap
|
||||
private val logRetentionMs = config.logRetentionHoursMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) // convert hours to ms
|
||||
private val logRetentionSize = config.logRetentionSize
|
||||
this.logIdent = "Log Manager on Broker " + config.brokerId + ", "
|
||||
|
||||
/* Initialize a log for each subdirectory of the main log directory */
|
||||
private val logs = new Pool[String, Pool[Int, Log]]()
|
||||
|
@ -60,11 +61,11 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
warn("Skipping unexplainable file '" + dir.getAbsolutePath() + "'--should it be there?")
|
||||
} else {
|
||||
info("Loading log '" + dir.getName() + "'")
|
||||
val log = new Log(dir, maxSize, flushInterval, needRecovery)
|
||||
val topicPartion = Utils.getTopicPartition(dir.getName)
|
||||
logs.putIfNotExists(topicPartion._1, new Pool[Int, Log]())
|
||||
val parts = logs.get(topicPartion._1)
|
||||
parts.put(topicPartion._2, log)
|
||||
val log = new Log(dir, maxSize, flushInterval, needRecovery, config.brokerId)
|
||||
val topicPartition = Utils.getTopicPartition(dir.getName)
|
||||
logs.putIfNotExists(topicPartition._1, new Pool[Int, Log]())
|
||||
val parts = logs.get(topicPartition._1)
|
||||
parts.put(topicPartition._2, log)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -78,9 +79,9 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
info("Starting log cleaner every " + logCleanupIntervalMs + " ms")
|
||||
scheduler.scheduleWithRate(cleanupLogs, "kafka-logcleaner-", 60 * 1000, logCleanupIntervalMs, false)
|
||||
info("Starting log flusher every " + config.flushSchedulerThreadRate +
|
||||
" ms with the following overrides " + logFlushIntervals)
|
||||
" ms with the following overrides " + logFlushIntervals)
|
||||
scheduler.scheduleWithRate(flushAllLogs, "kafka-logflusher-",
|
||||
config.flushSchedulerThreadRate, config.flushSchedulerThreadRate, false)
|
||||
config.flushSchedulerThreadRate, config.flushSchedulerThreadRate, false)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,14 +94,14 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
throw new InvalidTopicException("Topic name can't be emtpy")
|
||||
if (partition < 0 || partition >= config.topicPartitionsMap.getOrElse(topic, numPartitions)) {
|
||||
val error = "Wrong partition %d, valid partitions (0, %d)."
|
||||
.format(partition, (config.topicPartitionsMap.getOrElse(topic, numPartitions) - 1))
|
||||
.format(partition, (config.topicPartitionsMap.getOrElse(topic, numPartitions) - 1))
|
||||
warn(error)
|
||||
throw new InvalidPartitionException(error)
|
||||
}
|
||||
logCreationLock synchronized {
|
||||
val d = new File(logDir, topic + "-" + partition)
|
||||
d.mkdirs()
|
||||
new Log(d, maxSize, flushInterval, false)
|
||||
new Log(d, maxSize, flushInterval, false, config.brokerId)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -195,18 +196,19 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
debug("Garbage collecting '" + log.name + "'")
|
||||
total += cleanupExpiredSegments(log) + cleanupSegmentsToMaintainSize(log)
|
||||
}
|
||||
debug("Log cleanup completed. " + total + " files deleted in " +
|
||||
(time.milliseconds - startMs) / 1000 + " seconds")
|
||||
debug("Log cleanup completed. " + total + " files deleted in " +
|
||||
(time.milliseconds - startMs) / 1000 + " seconds")
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Close all the logs
|
||||
*/
|
||||
def shutdown() {
|
||||
info("Closing log manager")
|
||||
info("shut down")
|
||||
allLogs.foreach(_.close())
|
||||
info("shutted down completedly")
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get all the partition logs
|
||||
*/
|
||||
|
@ -222,7 +224,7 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
|||
if(logFlushIntervals.contains(log.topicName))
|
||||
logFlushInterval = logFlushIntervals(log.topicName)
|
||||
debug(log.topicName + " flush interval " + logFlushInterval +
|
||||
" last flushed " + log.getLastFlushedTime + " timesincelastFlush: " + timeSinceLastFlush)
|
||||
" last flushed " + log.getLastFlushedTime + " timesincelastFlush: " + timeSinceLastFlush)
|
||||
if(timeSinceLastFlush >= logFlushInterval)
|
||||
log.flush
|
||||
}
|
||||
|
|
|
@ -31,12 +31,13 @@ import kafka.utils._
|
|||
* N Processor threads that each have their own selector and read requests from sockets
|
||||
* M Handler threads that handle requests and produce responses back to the processor threads for writing.
|
||||
*/
|
||||
class SocketServer(val port: Int,
|
||||
class SocketServer(val brokerId: Int,
|
||||
val port: Int,
|
||||
val numProcessorThreads: Int,
|
||||
val monitoringPeriodSecs: Int,
|
||||
val maxQueuedRequests: Int,
|
||||
val maxRequestSize: Int = Int.MaxValue) extends Logging {
|
||||
|
||||
this.logIdent = "Socket Server on Broker " + brokerId + ", "
|
||||
private val time = SystemTime
|
||||
private val processors = new Array[Processor](numProcessorThreads)
|
||||
private var acceptor: Acceptor = new Acceptor(port, processors)
|
||||
|
@ -57,18 +58,18 @@ class SocketServer(val port: Int,
|
|||
// start accepting connections
|
||||
Utils.newThread("kafka-acceptor", acceptor, false).start()
|
||||
acceptor.awaitStartup
|
||||
info("Kafka socket server started")
|
||||
info("started")
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown the socket server
|
||||
*/
|
||||
def shutdown() = {
|
||||
info("Shutting down socket server")
|
||||
info("shutting down")
|
||||
acceptor.shutdown
|
||||
for(processor <- processors)
|
||||
processor.shutdown
|
||||
info("Shut down socket server complete")
|
||||
info("shutted down completely")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ import kafka.cluster.Broker
|
|||
|
||||
abstract class AbstractFetcherManager(protected val name: String, numFetchers: Int = 1) extends Logging {
|
||||
// map of (source brokerid, fetcher Id per source broker) => fetcher
|
||||
private val fetcherThreadMap = new mutable.HashMap[Tuple2[Int, Int], AbstractFetcherThread]
|
||||
private val fetcherThreadMap = new mutable.HashMap[(Int, Int), AbstractFetcherThread]
|
||||
private val mapLock = new Object
|
||||
this.logIdent = name + " "
|
||||
|
||||
|
@ -52,7 +52,7 @@ abstract class AbstractFetcherManager(protected val name: String, numFetchers: I
|
|||
}
|
||||
|
||||
def removeFetcher(topic: String, partitionId: Int) {
|
||||
info("%s removing fetcher on topic %s, partition %d".format(name, topic, partitionId))
|
||||
info("removing fetcher on topic %s, partition %d".format(topic, partitionId))
|
||||
mapLock synchronized {
|
||||
for ((key, fetcher) <- fetcherThreadMap) {
|
||||
fetcher.removePartition(topic, partitionId)
|
||||
|
|
|
@ -25,24 +25,29 @@ import kafka.common._
|
|||
import kafka.log._
|
||||
import kafka.message._
|
||||
import kafka.network._
|
||||
import kafka.utils.{SystemTime, Logging}
|
||||
import org.apache.log4j.Logger
|
||||
import scala.collection._
|
||||
import mutable.HashMap
|
||||
import scala.math._
|
||||
import kafka.network.RequestChannel.Response
|
||||
import kafka.utils.{ZkUtils, SystemTime, Logging}
|
||||
import kafka.cluster.Replica
|
||||
|
||||
/**
|
||||
* Logic to handle the various Kafka requests
|
||||
*/
|
||||
class KafkaApis(val requestChannel: RequestChannel,
|
||||
val logManager: LogManager,
|
||||
val replicaManager: ReplicaManager,
|
||||
val kafkaZookeeper: KafkaZooKeeper) extends Logging {
|
||||
class KafkaApis(val requestChannel: RequestChannel, val logManager: LogManager,
|
||||
val replicaManager: ReplicaManager, val kafkaZookeeper: KafkaZooKeeper,
|
||||
addReplicaCbk: (String, Int, Set[Int]) => Replica,
|
||||
stopReplicaCbk: (String, Int) => Short,
|
||||
becomeLeader: (Replica, LeaderAndISR) => Short,
|
||||
becomeFollower: (Replica, LeaderAndISR) => Short,
|
||||
brokerId: Int) extends Logging {
|
||||
|
||||
private val produceRequestPurgatory = new ProducerRequestPurgatory
|
||||
private val fetchRequestPurgatory = new FetchRequestPurgatory(requestChannel)
|
||||
private val produceRequestPurgatory = new ProducerRequestPurgatory(brokerId)
|
||||
private val fetchRequestPurgatory = new FetchRequestPurgatory(brokerId, requestChannel)
|
||||
private val requestLogger = Logger.getLogger("kafka.request.logger")
|
||||
this.logIdent = "KafkaApi on Broker " + brokerId + ", "
|
||||
|
||||
/**
|
||||
* Top-level method that handles all requests and multiplexes to the right api
|
||||
|
@ -62,12 +67,46 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
|
||||
|
||||
def handleLeaderAndISRRequest(request: RequestChannel.Request){
|
||||
val leaderAndISRRequest = LeaderAndISRRequest.readFrom(request.request.buffer)
|
||||
val responseMap = new HashMap[(String, Int), Short]
|
||||
val leaderAndISRRequest = LeaderAndISRRequest.readFrom(request.request.buffer)
|
||||
info("handling leader and isr request " + leaderAndISRRequest)
|
||||
|
||||
// TODO: put in actual logic later
|
||||
for((key, value) <- leaderAndISRRequest.leaderAndISRInfos){
|
||||
responseMap.put(key, ErrorMapping.NoError)
|
||||
for((partitionInfo, leaderAndISR) <- leaderAndISRRequest.leaderAndISRInfos){
|
||||
var errorCode = ErrorMapping.NoError
|
||||
val topic = partitionInfo._1
|
||||
val partition = partitionInfo._2
|
||||
|
||||
// If the partition does not exist locally, create it
|
||||
if(replicaManager.getPartition(topic, partition) == None){
|
||||
trace("the partition (%s, %d) does not exist locally, check if current broker is in assigned replicas, if so, start the local replica".format(topic, partition))
|
||||
val assignedReplicas = ZkUtils.getReplicasForPartition(kafkaZookeeper.getZookeeperClient, topic, partition)
|
||||
trace("assigned replicas list for topic [%s] partition [%d] is [%s]".format(topic, partition, assignedReplicas.toString))
|
||||
if(assignedReplicas.contains(brokerId)) {
|
||||
val replica = addReplicaCbk(topic, partition, assignedReplicas.toSet)
|
||||
info("starting replica for topic [%s] partition [%d]".format(replica.topic, replica.partition.partitionId))
|
||||
}
|
||||
}
|
||||
val replica = replicaManager.getReplica(topic, partition).get
|
||||
// The command ask this broker to be new leader for P and it isn't the leader yet
|
||||
val requestedLeaderId = leaderAndISR.leader
|
||||
// If the broker is requested to be the leader and it's not current the leader (the leader id is set and not equal to broker id)
|
||||
if(requestedLeaderId == brokerId && (!replica.partition.leaderId().isDefined || replica.partition.leaderId().get != brokerId)){
|
||||
info("becoming the leader for partition [%s, %d] at the leader and isr request %s".format(topic, partition, leaderAndISRRequest))
|
||||
errorCode = becomeLeader(replica, leaderAndISR)
|
||||
}
|
||||
else if (requestedLeaderId != brokerId) {
|
||||
info("becoming the follower for partition [%s, %d] at the leader and isr request %s".format(topic, partition, leaderAndISRRequest))
|
||||
errorCode = becomeFollower(replica, leaderAndISR)
|
||||
}
|
||||
|
||||
responseMap.put(partitionInfo, errorCode)
|
||||
}
|
||||
|
||||
if(leaderAndISRRequest.isInit == LeaderAndISRRequest.IsInit){
|
||||
replicaManager.startHighWaterMarksCheckPointThread
|
||||
val partitionsToRemove = replicaManager.allPartitions.filter(p => !leaderAndISRRequest.leaderAndISRInfos.contains(p._1)).keySet
|
||||
info("init flag is set in leaderAndISR request, partitions to remove: %s".format(partitionsToRemove))
|
||||
partitionsToRemove.foreach(p => stopReplicaCbk(p._1, p._2))
|
||||
}
|
||||
|
||||
val leaderAndISRResponse = new LeaderAndISRResponse(leaderAndISRRequest.versionId, responseMap)
|
||||
|
@ -79,9 +118,9 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val stopReplicaRequest = StopReplicaRequest.readFrom(request.request.buffer)
|
||||
val responseMap = new HashMap[(String, Int), Short]
|
||||
|
||||
// TODO: put in actual logic later
|
||||
for((topic, partition) <- stopReplicaRequest.stopReplicaSet){
|
||||
responseMap.put((topic, partition), ErrorMapping.NoError)
|
||||
val errorCode = stopReplicaCbk(topic, partition)
|
||||
responseMap.put((topic, partition), errorCode)
|
||||
}
|
||||
val stopReplicaResponse = new StopReplicaResponse(stopReplicaRequest.versionId, responseMap)
|
||||
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(stopReplicaResponse)))
|
||||
|
@ -95,7 +134,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
var satisfied = new mutable.ArrayBuffer[DelayedFetch]
|
||||
for(partitionData <- partitionDatas)
|
||||
satisfied ++= fetchRequestPurgatory.update((topic, partitionData.partition), partitionData)
|
||||
trace("Produce request to %s unblocked %d DelayedFetchRequests.".format(topic, satisfied.size))
|
||||
trace("produce request to %s unblocked %d DelayedFetchRequests.".format(topic, satisfied.size))
|
||||
// send any newly unblocked responses
|
||||
for(fetchReq <- satisfied) {
|
||||
val topicData = readMessageSets(fetchReq.fetch)
|
||||
|
@ -111,11 +150,11 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val produceRequest = ProducerRequest.readFrom(request.request.buffer)
|
||||
val sTime = SystemTime.milliseconds
|
||||
if(requestLogger.isTraceEnabled)
|
||||
requestLogger.trace("Producer request " + request.toString)
|
||||
requestLogger.trace("producer request %s".format(produceRequest.toString))
|
||||
trace("Broker %s received produce request %s".format(logManager.config.brokerId, produceRequest.toString))
|
||||
|
||||
val response = produceToLocalLog(produceRequest)
|
||||
debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime))
|
||||
debug("produce to local log in %d ms".format(SystemTime.milliseconds - sTime))
|
||||
|
||||
if (produceRequest.requiredAcks == 0 ||
|
||||
produceRequest.requiredAcks == 1 ||
|
||||
|
@ -133,13 +172,11 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
(topic, partitionData.partition)
|
||||
})
|
||||
})
|
||||
|
||||
val delayedProduce = new DelayedProduce(
|
||||
topicPartitionPairs, request,
|
||||
response.errors, response.offsets,
|
||||
produceRequest, produceRequest.ackTimeoutMs.toLong)
|
||||
produceRequestPurgatory.watch(delayedProduce)
|
||||
|
||||
/*
|
||||
* Replica fetch requests may have arrived (and potentially satisfied)
|
||||
* delayedProduce requests before they even made it to the purgatory.
|
||||
|
@ -147,10 +184,9 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
*/
|
||||
var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce]
|
||||
topicPartitionPairs.foreach(topicPartition =>
|
||||
satisfiedProduceRequests ++=
|
||||
produceRequestPurgatory.update(topicPartition, topicPartition))
|
||||
debug(satisfiedProduceRequests.size +
|
||||
" DelayedProduce requests unblocked after produce to local log.")
|
||||
satisfiedProduceRequests ++=
|
||||
produceRequestPurgatory.update(topicPartition, topicPartition))
|
||||
debug("%d DelayedProduce requests unblocked after produce to local log.".format(satisfiedProduceRequests.size))
|
||||
satisfiedProduceRequests.foreach(_.respond())
|
||||
}
|
||||
}
|
||||
|
@ -159,10 +195,10 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
* Helper method for handling a parsed producer request
|
||||
*/
|
||||
private def produceToLocalLog(request: ProducerRequest): ProducerResponse = {
|
||||
trace("produce [%s] to local log ".format(request.toString))
|
||||
val requestSize = request.topicPartitionCount
|
||||
val errors = new Array[Short](requestSize)
|
||||
val offsets = new Array[Long](requestSize)
|
||||
|
||||
var msgIndex = -1
|
||||
for(topicData <- request.data) {
|
||||
for(partitionData <- topicData.partitionDataArray) {
|
||||
|
@ -181,7 +217,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
case e =>
|
||||
BrokerTopicStat.getBrokerTopicStat(topicData.topic).recordFailedProduceRequest
|
||||
BrokerTopicStat.getBrokerAllTopicStat.recordFailedProduceRequest
|
||||
error("Error processing ProducerRequest on " + topicData.topic + ":" + partitionData.partition, e)
|
||||
error("error processing ProducerRequest on " + topicData.topic + ":" + partitionData.partition, e)
|
||||
e match {
|
||||
case _: IOException =>
|
||||
fatal("Halting due to unrecoverable I/O error while handling producer request: " + e.getMessage, e)
|
||||
|
@ -193,7 +229,8 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
}
|
||||
}
|
||||
new ProducerResponse(request.versionId, request.correlationId, errors, offsets)
|
||||
val ret = new ProducerResponse(request.versionId, request.correlationId, errors, offsets)
|
||||
ret
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -201,9 +238,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
*/
|
||||
def handleFetchRequest(request: RequestChannel.Request) {
|
||||
val fetchRequest = FetchRequest.readFrom(request.request.buffer)
|
||||
if(requestLogger.isTraceEnabled)
|
||||
requestLogger.trace("Fetch request " + fetchRequest.toString)
|
||||
trace("Broker %s received fetch request %s".format(logManager.config.brokerId, fetchRequest.toString))
|
||||
trace("handling fetch request: " + fetchRequest.toString)
|
||||
// validate the request
|
||||
try {
|
||||
fetchRequest.validate()
|
||||
|
@ -225,8 +260,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
)
|
||||
})
|
||||
})
|
||||
trace("Replica %d fetch unblocked %d DelayedProduce requests.".format(
|
||||
fetchRequest.replicaId, satisfiedProduceRequests.size))
|
||||
debug("replica %d fetch unblocked %d DelayedProduce requests.".format(fetchRequest.replicaId, satisfiedProduceRequests.size))
|
||||
satisfiedProduceRequests.foreach(_.respond())
|
||||
}
|
||||
|
||||
|
@ -236,11 +270,11 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
availableBytes >= fetchRequest.minBytes ||
|
||||
fetchRequest.numPartitions <= 0) {
|
||||
val topicData = readMessageSets(fetchRequest)
|
||||
debug("Returning fetch response %s for fetch request with correlation id %d"
|
||||
.format(topicData.map(_.partitionDataArray.map(_.error).mkString(",")).mkString(","), fetchRequest.correlationId))
|
||||
debug("returning fetch response %s for fetch request with correlation id %d".format(topicData.map(_.partitionDataArray.map(_.error).mkString(",")).mkString(","), fetchRequest.correlationId))
|
||||
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, topicData)
|
||||
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
|
||||
} else {
|
||||
debug("putting fetch request into purgatory")
|
||||
// create a list of (topic, partition) pairs to use as keys for this delayed request
|
||||
val topicPartitionPairs: Seq[Any] = fetchRequest.offsetInfo.flatMap(o => o.partitions.map((o.topic, _)))
|
||||
val delayedFetch = new DelayedFetch(topicPartitionPairs, request, fetchRequest, fetchRequest.maxWait, availableBytes)
|
||||
|
@ -256,7 +290,6 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
for(offsetDetail <- fetchRequest.offsetInfo) {
|
||||
for(i <- 0 until offsetDetail.partitions.size) {
|
||||
try {
|
||||
debug("Fetching log for topic %s partition %d".format(offsetDetail.topic, offsetDetail.partitions(i)))
|
||||
val maybeLog = logManager.getLog(offsetDetail.topic, offsetDetail.partitions(i))
|
||||
val available = maybeLog match {
|
||||
case Some(log) => max(0, log.logEndOffset - offsetDetail.offsets(i))
|
||||
|
@ -265,7 +298,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
totalBytes += math.min(offsetDetail.fetchSizes(i), available)
|
||||
} catch {
|
||||
case e: InvalidPartitionException =>
|
||||
info("Invalid partition " + offsetDetail.partitions(i) + "in fetch request from client '" + fetchRequest.clientId + "'")
|
||||
info("invalid partition " + offsetDetail.partitions(i) + "in fetch request from client '" + fetchRequest.clientId + "'")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -274,13 +307,13 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
|
||||
private def maybeUpdatePartitionHW(fetchRequest: FetchRequest) {
|
||||
val offsets = fetchRequest.offsetInfo
|
||||
|
||||
debug("act on update partition HW, check offset detail: %s ".format(offsets))
|
||||
for(offsetDetail <- offsets) {
|
||||
val topic = offsetDetail.topic
|
||||
val (partitions, offsets) = (offsetDetail.partitions, offsetDetail.offsets)
|
||||
for( (partition, offset) <- (partitions, offsets).zipped.map((_,_))) {
|
||||
replicaManager.recordFollowerPosition(topic, partition, fetchRequest.replicaId, offset,
|
||||
kafkaZookeeper.getZookeeperClient)
|
||||
kafkaZookeeper.getZookeeperClient)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -310,21 +343,17 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
BrokerTopicStat.getBrokerTopicStat(topic).recordBytesOut(messages.sizeInBytes)
|
||||
BrokerTopicStat.getBrokerAllTopicStat.recordBytesOut(messages.sizeInBytes)
|
||||
val leaderReplicaOpt = replicaManager.getReplica(topic, partition, logManager.config.brokerId)
|
||||
assert(leaderReplicaOpt.isDefined, "Leader replica for topic %s partition %d".format(topic, partition) +
|
||||
" must exist on leader broker %d".format(logManager.config.brokerId))
|
||||
assert(leaderReplicaOpt.isDefined, "Leader replica for topic %s partition %d".format(topic, partition) + " must exist on leader broker %d".format(logManager.config.brokerId))
|
||||
val leaderReplica = leaderReplicaOpt.get
|
||||
fetchRequest.replicaId match {
|
||||
case FetchRequest.NonFollowerId => // replica id value of -1 signifies a fetch request from an external client, not from one of the replicas
|
||||
new PartitionData(partition, ErrorMapping.NoError, offset, leaderReplica.highWatermark(), messages)
|
||||
case _ => // fetch request from a follower
|
||||
val replicaOpt = replicaManager.getReplica(topic, partition, fetchRequest.replicaId)
|
||||
assert(replicaOpt.isDefined, "No replica %d in replica manager on %d"
|
||||
.format(fetchRequest.replicaId, replicaManager.config.brokerId))
|
||||
assert(replicaOpt.isDefined, "No replica %d in replica manager on %d".format(fetchRequest.replicaId, replicaManager.config.brokerId))
|
||||
val replica = replicaOpt.get
|
||||
debug("Leader %d for topic %s partition %d received fetch request from follower %d"
|
||||
.format(logManager.config.brokerId, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
||||
debug("Leader %d returning %d messages for topic %s partition %d to follower %d"
|
||||
.format(logManager.config.brokerId, messages.sizeInBytes, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
||||
debug("leader [%d] for topic [%s] partition [%d] received fetch request from follower [%d]".format(logManager.config.brokerId, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
||||
debug("Leader %d returning %d messages for topic %s partition %d to follower %d".format(logManager.config.brokerId, messages.sizeInBytes, replica.topic, replica.partition.partitionId, fetchRequest.replicaId))
|
||||
new PartitionData(partition, ErrorMapping.NoError, offset, leaderReplica.highWatermark(), messages)
|
||||
}
|
||||
}
|
||||
|
@ -343,7 +372,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
try {
|
||||
// check if the current broker is the leader for the partitions
|
||||
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(topic, partition)
|
||||
trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize))
|
||||
trace("fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize))
|
||||
val log = logManager.getLog(topic, partition)
|
||||
response = Right(log match { case Some(l) => l.read(offset, maxSize) case None => MessageSet.Empty })
|
||||
} catch {
|
||||
|
@ -360,7 +389,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
def handleOffsetRequest(request: RequestChannel.Request) {
|
||||
val offsetRequest = OffsetRequest.readFrom(request.request.buffer)
|
||||
if(requestLogger.isTraceEnabled)
|
||||
requestLogger.trace("Offset request " + offsetRequest.toString)
|
||||
requestLogger.trace("offset request " + offsetRequest.toString)
|
||||
var response: OffsetResponse = null
|
||||
try {
|
||||
kafkaZookeeper.ensurePartitionLeaderOnThisBroker(offsetRequest.topic, offsetRequest.partition)
|
||||
|
@ -372,8 +401,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
System.exit(1)
|
||||
case e =>
|
||||
warn("Error while responding to offset request", e)
|
||||
response = new OffsetResponse(offsetRequest.versionId, Array.empty[Long],
|
||||
ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]).toShort)
|
||||
response = new OffsetResponse(offsetRequest.versionId, Array.empty[Long],ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]).toShort)
|
||||
}
|
||||
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
|
||||
}
|
||||
|
@ -384,41 +412,39 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
def handleTopicMetadataRequest(request: RequestChannel.Request) {
|
||||
val metadataRequest = TopicMetadataRequest.readFrom(request.request.buffer)
|
||||
if(requestLogger.isTraceEnabled)
|
||||
requestLogger.trace("Topic metadata request " + metadataRequest.toString())
|
||||
|
||||
requestLogger.trace("topic metadata request " + metadataRequest.toString())
|
||||
val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]()
|
||||
val zkClient = kafkaZookeeper.getZookeeperClient
|
||||
var errorCode = ErrorMapping.NoError
|
||||
val config = logManager.config
|
||||
|
||||
try {
|
||||
val topicMetadataList = AdminUtils.getTopicMetaDataFromZK(metadataRequest.topics, zkClient)
|
||||
|
||||
metadataRequest.topics.zip(topicMetadataList).foreach { topicAndMetadata =>
|
||||
val topic = topicAndMetadata._1
|
||||
topicAndMetadata._2.errorCode match {
|
||||
case ErrorMapping.NoError => topicsMetadata += topicAndMetadata._2
|
||||
case ErrorMapping.UnknownTopicCode =>
|
||||
/* check if auto creation of topics is turned on */
|
||||
if(config.autoCreateTopics) {
|
||||
CreateTopicCommand.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor)
|
||||
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
|
||||
.format(topic, config.numPartitions, config.defaultReplicationFactor))
|
||||
val newTopicMetadata = AdminUtils.getTopicMetaDataFromZK(List(topic), zkClient).head
|
||||
newTopicMetadata.errorCode match {
|
||||
case ErrorMapping.NoError => topicsMetadata += newTopicMetadata
|
||||
case _ =>
|
||||
throw new KafkaException("Topic metadata for automatically created topic %s does not exist".format(topic))
|
||||
metadataRequest.topics.zip(topicMetadataList).foreach(
|
||||
topicAndMetadata =>{
|
||||
val topic = topicAndMetadata._1
|
||||
topicAndMetadata._2.errorCode match {
|
||||
case ErrorMapping.NoError => topicsMetadata += topicAndMetadata._2
|
||||
case ErrorMapping.UnknownTopicCode =>
|
||||
/* check if auto creation of topics is turned on */
|
||||
if(config.autoCreateTopics) {
|
||||
CreateTopicCommand.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor)
|
||||
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
|
||||
.format(topic, config.numPartitions, config.defaultReplicationFactor))
|
||||
val newTopicMetadata = AdminUtils.getTopicMetaDataFromZK(List(topic), zkClient).head
|
||||
newTopicMetadata.errorCode match {
|
||||
case ErrorMapping.NoError => topicsMetadata += newTopicMetadata
|
||||
case _ =>
|
||||
throw new KafkaException("Topic metadata for automatically created topic %s does not exist".format(topic))
|
||||
}
|
||||
}
|
||||
}
|
||||
case _ => error("Error while fetching topic metadata for topic " + topic,
|
||||
ErrorMapping.exceptionFor(topicAndMetadata._2.errorCode).getCause)
|
||||
}
|
||||
}
|
||||
case _ => error("Error while fetching topic metadata for topic " + topic,
|
||||
ErrorMapping.exceptionFor(topicAndMetadata._2.errorCode).getCause)
|
||||
}
|
||||
})
|
||||
}catch {
|
||||
case e => error("Error while retrieving topic metadata", e)
|
||||
// convert exception type to error code
|
||||
errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])
|
||||
// convert exception type to error code
|
||||
errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])
|
||||
}
|
||||
topicsMetadata.foreach(metadata => trace("Sending topic metadata " + metadata.toString))
|
||||
val response = new TopicMetaDataResponse(metadataRequest.versionId, topicsMetadata.toSeq, errorCode)
|
||||
|
@ -426,7 +452,10 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
|
||||
def close() {
|
||||
debug("shut down")
|
||||
fetchRequestPurgatory.shutdown()
|
||||
produceRequestPurgatory.shutdown()
|
||||
debug("shutted down completely")
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -439,7 +468,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
/**
|
||||
* A holding pen for fetch requests waiting to be satisfied
|
||||
*/
|
||||
class FetchRequestPurgatory(requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, PartitionData] {
|
||||
class FetchRequestPurgatory(brokerId: Int, requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, PartitionData]("Fetch Request Purgatory on Broker " + brokerId + ", ") {
|
||||
|
||||
/**
|
||||
* A fetch request is satisfied when it has accumulated enough data to meet the min_bytes field
|
||||
|
@ -447,6 +476,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
def checkSatisfied(partitionData: PartitionData, delayedFetch: DelayedFetch): Boolean = {
|
||||
val messageDataSize = partitionData.messages.sizeInBytes
|
||||
val accumulatedSize = delayedFetch.bytesAccumulated.addAndGet(messageDataSize)
|
||||
debug("fetch request check, accm size: " + accumulatedSize + " delay fetch min bytes: " + delayedFetch.fetch.minBytes)
|
||||
accumulatedSize >= delayedFetch.fetch.minBytes
|
||||
}
|
||||
|
||||
|
@ -466,7 +496,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
requiredOffsets: Array[Long],
|
||||
val produce: ProducerRequest,
|
||||
delayMs: Long)
|
||||
extends DelayedRequest(keys, request, delayMs) with Logging {
|
||||
extends DelayedRequest(keys, request, delayMs) with Logging {
|
||||
|
||||
/**
|
||||
* Map of (topic, partition) -> partition status
|
||||
|
@ -495,15 +525,15 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
|
||||
def respond() {
|
||||
val errorsAndOffsets: (List[Short], List[Long]) = (
|
||||
keys.foldRight
|
||||
((List[Short](), List[Long]()))
|
||||
((key: Any, result: (List[Short], List[Long])) => {
|
||||
val status = partitionStatus(key)
|
||||
(status.error :: result._1, status.requiredOffset :: result._2)
|
||||
})
|
||||
)
|
||||
keys.foldRight
|
||||
((List[Short](), List[Long]()))
|
||||
((key: Any, result: (List[Short], List[Long])) => {
|
||||
val status = partitionStatus(key)
|
||||
(status.error :: result._1, status.requiredOffset :: result._2)
|
||||
})
|
||||
)
|
||||
val response = new ProducerResponse(produce.versionId, produce.correlationId,
|
||||
errorsAndOffsets._1.toArray, errorsAndOffsets._2.toArray)
|
||||
errorsAndOffsets._1.toArray, errorsAndOffsets._2.toArray)
|
||||
|
||||
requestChannel.sendResponse(new RequestChannel.Response(
|
||||
request, new BoundedByteBufferSend(response)))
|
||||
|
@ -539,7 +569,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
numAcks, produce.requiredAcks,
|
||||
topic, partitionId))
|
||||
if ((produce.requiredAcks < 0 && numAcks >= isr.size) ||
|
||||
(produce.requiredAcks > 0 && numAcks >= produce.requiredAcks)) {
|
||||
(produce.requiredAcks > 0 && numAcks >= produce.requiredAcks)) {
|
||||
/*
|
||||
* requiredAcks < 0 means acknowledge after all replicas in ISR
|
||||
* are fully caught up to the (local) leader's offset
|
||||
|
@ -588,8 +618,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
/**
|
||||
* A holding pen for produce requests waiting to be satisfied.
|
||||
*/
|
||||
private [kafka] class ProducerRequestPurgatory
|
||||
extends RequestPurgatory[DelayedProduce, (String, Int)] {
|
||||
private [kafka] class ProducerRequestPurgatory(brokerId: Int) extends RequestPurgatory[DelayedProduce, (String, Int)]("Producer Request Purgatory on Broker " + brokerId + ", ") {
|
||||
|
||||
protected def checkSatisfied(fetchRequestPartition: (String, Int),
|
||||
delayedProduce: DelayedProduce) =
|
||||
|
|
|
@ -16,12 +16,11 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import kafka.common.KafkaZookeeperClient
|
||||
import collection.mutable.HashMap
|
||||
import collection._
|
||||
import collection.immutable.Set
|
||||
import kafka.cluster.Broker
|
||||
import kafka.api._
|
||||
import java.lang.Object
|
||||
import kafka.network.{Receive, BlockingChannel}
|
||||
import kafka.utils.{ZkUtils, Logging}
|
||||
import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue, BlockingQueue}
|
||||
|
@ -29,36 +28,46 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException
|
|||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import org.I0Itec.zkclient.{IZkStateListener, ZkClient, IZkDataListener, IZkChildListener}
|
||||
import org.apache.zookeeper.Watcher.Event.KeeperState
|
||||
import collection.JavaConversions._
|
||||
import java.lang.Object
|
||||
import java.nio.channels.AsynchronousCloseException
|
||||
|
||||
|
||||
class RequestSendThread(val brokerId: Int,
|
||||
class RequestSendThread(val controllerId: Int,
|
||||
val toBrokerId: Int,
|
||||
val queue: BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)],
|
||||
val channel: BlockingChannel)
|
||||
extends Thread("requestSendThread-" + brokerId) with Logging {
|
||||
extends Thread("requestSendThread-" + toBrokerId) with Logging {
|
||||
this.logIdent = "Controller %d, request send thread to broker %d, ".format(controllerId, toBrokerId)
|
||||
val isRunning: AtomicBoolean = new AtomicBoolean(true)
|
||||
private val shutDownLatch = new CountDownLatch(1)
|
||||
private val lock = new Object
|
||||
private val lock = new Object()
|
||||
|
||||
def shutDown(): Unit = {
|
||||
info("Shutting down controller request send thread to broker %d".format(brokerId))
|
||||
info("shutting down")
|
||||
isRunning.set(false)
|
||||
interrupt()
|
||||
shutDownLatch.await()
|
||||
info("Controller request send thread to broker %d shutting down completed".format(brokerId))
|
||||
info("shutted down completed")
|
||||
}
|
||||
|
||||
override def run(): Unit = {
|
||||
try{
|
||||
info("In controller, thread for broker: " + brokerId + " started running")
|
||||
while(isRunning.get()){
|
||||
val queueItem = queue.take()
|
||||
val request = queueItem._1
|
||||
val callback = queueItem._2
|
||||
|
||||
var receive: Receive = null
|
||||
lock synchronized {
|
||||
channel.send(request)
|
||||
receive = channel.receive()
|
||||
try{
|
||||
lock synchronized {
|
||||
channel.send(request)
|
||||
receive = channel.receive()
|
||||
}
|
||||
} catch {
|
||||
case e =>
|
||||
// log it and let it go. Let controller shut it down.
|
||||
debug("Exception occurs", e)
|
||||
}
|
||||
|
||||
var response: RequestOrResponse = null
|
||||
|
@ -68,13 +77,15 @@ class RequestSendThread(val brokerId: Int,
|
|||
case RequestKeys.StopReplicaRequest =>
|
||||
response = StopReplicaResponse.readFrom(receive.buffer)
|
||||
}
|
||||
trace("got a response %s".format(controllerId, response, toBrokerId))
|
||||
|
||||
if(callback != null){
|
||||
callback(response)
|
||||
}
|
||||
}
|
||||
} catch{
|
||||
case e: InterruptedException => warn("Controller request send thread to broker %d is intterrupted. Shutting down".format(brokerId))
|
||||
case e1 => error("Error in controller request send thread to broker %d down due to ".format(brokerId), e1)
|
||||
case e: InterruptedException => warn("intterrupted. Shutting down")
|
||||
case e1 => error("Error due to ", e1)
|
||||
}
|
||||
shutDownLatch.countDown()
|
||||
}
|
||||
|
@ -85,9 +96,10 @@ class ControllerChannelManager(allBrokers: Set[Broker], config : KafkaConfig) ex
|
|||
private val messageChannels = new HashMap[Int, BlockingChannel]
|
||||
private val messageQueues = new HashMap[Int, BlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)]]
|
||||
private val messageThreads = new HashMap[Int, RequestSendThread]
|
||||
this.logIdent = "Channel manager on controller " + config.brokerId + ", "
|
||||
for(broker <- allBrokers){
|
||||
brokers.put(broker.id, broker)
|
||||
info("channel for broker " + broker.id + " created" + " at host: " + broker.host + " and port: " + broker.port)
|
||||
info("channel to broker " + broker.id + " created" + " at host: " + broker.host + " and port: " + broker.port)
|
||||
val channel = new BlockingChannel(broker.host, broker.port,
|
||||
BlockingChannel.UseDefaultBufferSize,
|
||||
BlockingChannel.UseDefaultBufferSize,
|
||||
|
@ -99,7 +111,7 @@ class ControllerChannelManager(allBrokers: Set[Broker], config : KafkaConfig) ex
|
|||
|
||||
def startUp() = {
|
||||
for((brokerId, broker) <- brokers){
|
||||
val thread = new RequestSendThread(brokerId, messageQueues(brokerId), messageChannels(brokerId))
|
||||
val thread = new RequestSendThread(config.brokerId, brokerId, messageQueues(brokerId), messageChannels(brokerId))
|
||||
thread.setDaemon(false)
|
||||
thread.start()
|
||||
messageThreads.put(broker.id, thread)
|
||||
|
@ -119,14 +131,13 @@ class ControllerChannelManager(allBrokers: Set[Broker], config : KafkaConfig) ex
|
|||
def addBroker(broker: Broker){
|
||||
brokers.put(broker.id, broker)
|
||||
messageQueues.put(broker.id, new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize))
|
||||
info("channel for broker " + broker.id + " created" + " at host: " + broker.host + " and port: " + broker.port)
|
||||
val channel = new BlockingChannel(broker.host, broker.port,
|
||||
BlockingChannel.UseDefaultBufferSize,
|
||||
BlockingChannel.UseDefaultBufferSize,
|
||||
config.controllerSocketTimeoutMs)
|
||||
channel.connect()
|
||||
messageChannels.put(broker.id, channel)
|
||||
val thread = new RequestSendThread(broker.id, messageQueues(broker.id), messageChannels(broker.id))
|
||||
val thread = new RequestSendThread(config.brokerId, broker.id, messageQueues(broker.id), messageChannels(broker.id))
|
||||
thread.setDaemon(false)
|
||||
thread.start()
|
||||
messageThreads.put(broker.id, thread)
|
||||
|
@ -146,38 +157,62 @@ class ControllerChannelManager(allBrokers: Set[Broker], config : KafkaConfig) ex
|
|||
}
|
||||
}
|
||||
|
||||
class KafkaController(config : KafkaConfig) extends Logging {
|
||||
info("controller startup");
|
||||
private val lock = new Object
|
||||
|
||||
private var zkClient: ZkClient = null
|
||||
class KafkaController(config : KafkaConfig, zkClient: ZkClient) extends Logging {
|
||||
this.logIdent = "Controller " + config.brokerId + ", "
|
||||
info("startup");
|
||||
private val controllerLock = new Object
|
||||
private var controllerChannelManager: ControllerChannelManager = null
|
||||
private var allBrokers : Set[Broker] = null
|
||||
private var allBrokerIds : Set[Int] = null
|
||||
private var allTopics: Set[String] = null
|
||||
private var allPartitionReplicaAssignment: mutable.Map[(String, Int), Seq[Int]] = null
|
||||
private var allLeaders: mutable.Map[(String, Int), Int] = null
|
||||
|
||||
private def tryToBecomeController() = {
|
||||
lock synchronized {
|
||||
val curController = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.ControllerPath)
|
||||
if (curController == null){
|
||||
try {
|
||||
ZkUtils.createEphemeralPathExpectConflict(zkClient, ZkUtils.ControllerPath, config.brokerId.toString())
|
||||
// Return true if this controller succeeds in the controller competition
|
||||
private def tryToBecomeController(): Boolean = {
|
||||
try {
|
||||
ZkUtils.createEphemeralPathExpectConflict(zkClient, ZkUtils.ControllerPath, config.brokerId.toString)
|
||||
// Only the broker successfully registering as the controller can execute following code, otherwise
|
||||
// some exception will be thrown.
|
||||
registerBrokerChangeListener()
|
||||
registerTopicChangeListener()
|
||||
allBrokers = ZkUtils.getAllBrokersInCluster(zkClient).toSet
|
||||
allBrokerIds = allBrokers.map(_.id)
|
||||
info("all brokers: %s".format(allBrokerIds))
|
||||
allTopics = ZkUtils.getAllTopics(zkClient).toSet
|
||||
info("all topics: %s".format(allTopics))
|
||||
allPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, allTopics.iterator)
|
||||
info("allPartitionReplicaAssignment: %s".format(allPartitionReplicaAssignment))
|
||||
allLeaders = new mutable.HashMap[(String, Int), Int]
|
||||
controllerChannelManager = new ControllerChannelManager(allBrokers, config)
|
||||
controllerChannelManager.startUp()
|
||||
return true
|
||||
} catch {
|
||||
case e: ZkNodeExistsException =>
|
||||
registerControllerExistListener()
|
||||
info("broker didn't succeed registering as the controller since it's taken by someone else")
|
||||
return false
|
||||
case e2 => throw e2
|
||||
}
|
||||
}
|
||||
|
||||
// Only the broker successfully registering as the controller can execute following code, otherwise
|
||||
// some exception will be thrown.
|
||||
registerBrokerChangeListener()
|
||||
registerTopicChangeListener()
|
||||
allBrokers = ZkUtils.getAllBrokersInCluster(zkClient).toSet
|
||||
allTopics = ZkUtils.getAllTopics(zkClient).toSet
|
||||
controllerChannelManager = new ControllerChannelManager(allBrokers, config)
|
||||
controllerChannelManager.startUp()
|
||||
} catch {
|
||||
case e: ZkNodeExistsException =>
|
||||
registerControllerExistListener()
|
||||
info("Broker " + config.brokerId + " didn't succeed registering as the controller since it's taken by someone else")
|
||||
case e2 => throw e2
|
||||
private def controllerRegisterOrFailover(){
|
||||
info("try to become controller")
|
||||
if(tryToBecomeController() == true){
|
||||
info("won the controller competition and work on leader and isr recovery")
|
||||
deliverLeaderAndISRFromZookeeper(allBrokerIds, allTopics)
|
||||
debug("work on broker changes")
|
||||
onBrokerChange()
|
||||
|
||||
// If there are some partition with leader not initialized, init the leader for them
|
||||
val partitionReplicaAssignment = allPartitionReplicaAssignment.clone()
|
||||
for((topicPartition, replicas) <- partitionReplicaAssignment){
|
||||
if (allLeaders.contains(topicPartition)){
|
||||
partitionReplicaAssignment.remove(topicPartition)
|
||||
}
|
||||
}
|
||||
else info("Broker " + config.brokerId + " see not null skip " + " current controller " + curController)
|
||||
debug("work on init leaders: %s, current cache for all leader is: %s".format(partitionReplicaAssignment.toString(), allLeaders))
|
||||
initLeaders(partitionReplicaAssignment)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -186,17 +221,22 @@ class KafkaController(config : KafkaConfig) extends Logging {
|
|||
}
|
||||
|
||||
def startup() = {
|
||||
zkClient = KafkaZookeeperClient.getZookeeperClient(config)
|
||||
registerSessionExpirationListener()
|
||||
registerControllerExistListener()
|
||||
tryToBecomeController()
|
||||
controllerLock synchronized {
|
||||
registerSessionExpirationListener()
|
||||
registerControllerExistListener()
|
||||
controllerRegisterOrFailover()
|
||||
}
|
||||
}
|
||||
|
||||
def shutDown() = {
|
||||
if(controllerChannelManager != null)
|
||||
controllerChannelManager.shutDown()
|
||||
if(zkClient != null)
|
||||
zkClient.close()
|
||||
controllerLock synchronized {
|
||||
if(controllerChannelManager != null){
|
||||
info("shut down")
|
||||
controllerChannelManager.shutDown()
|
||||
controllerChannelManager = null
|
||||
info("shutted down completely")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def sendRequest(brokerId : Int, request : RequestOrResponse, callback: (RequestOrResponse) => Unit = null) = {
|
||||
|
@ -219,7 +259,8 @@ class KafkaController(config : KafkaConfig) extends Logging {
|
|||
zkClient.subscribeDataChanges(ZkUtils.ControllerPath, new ControllerExistListener())
|
||||
}
|
||||
|
||||
class SessionExpireListener() extends IZkStateListener {
|
||||
class SessionExpireListener() extends IZkStateListener with Logging {
|
||||
this.logIdent = "Controller " + config.brokerId + ", "
|
||||
@throws(classOf[Exception])
|
||||
def handleStateChanged(state: KeeperState) {
|
||||
// do nothing, since zkclient will do reconnect for us.
|
||||
|
@ -234,50 +275,256 @@ class KafkaController(config : KafkaConfig) extends Logging {
|
|||
*/
|
||||
@throws(classOf[Exception])
|
||||
def handleNewSession() {
|
||||
info("Controller session expires, clean up the state, current controller: " + config.brokerId)
|
||||
controllerChannelManager.shutDown()
|
||||
controllerChannelManager = null
|
||||
info("Controller session expires, the channel manager shut downr: " + config.brokerId)
|
||||
tryToBecomeController()
|
||||
controllerLock synchronized {
|
||||
info("session expires, clean up the state")
|
||||
controllerChannelManager.shutDown()
|
||||
controllerChannelManager = null
|
||||
controllerRegisterOrFailover()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to populate the leaderAndISR from zookeeper to affected brokers when the brokers comes up
|
||||
*/
|
||||
private def deliverLeaderAndISRFromZookeeper(brokerIds: Set[Int], topics: Set[String]) = {
|
||||
val leaderAndISRInfos = ZkUtils.getPartitionLeaderAndISRForTopics(zkClient, topics.iterator)
|
||||
val brokerToLeaderAndISRInfosMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), LeaderAndISR]]
|
||||
for((topicPartition, leaderAndISR) <- leaderAndISRInfos){
|
||||
// If the leader specified in the leaderAndISR is no longer alive, there is no need to recover it
|
||||
if(allBrokerIds.contains(leaderAndISR.leader)){
|
||||
val brokersAssignedToThisPartitionOpt = allPartitionReplicaAssignment.get(topicPartition)
|
||||
if(brokersAssignedToThisPartitionOpt == None){
|
||||
warn("during leaderAndISR recovery, there's no replica assignment for partition [%s, %d] with allPartitionReplicaAssignment: %s".format(topicPartition._1, topicPartition._2, allPartitionReplicaAssignment))
|
||||
} else{
|
||||
val relatedBrokersAssignedToThisPartition = brokersAssignedToThisPartitionOpt.get.filter(brokerIds.contains(_))
|
||||
relatedBrokersAssignedToThisPartition.foreach(b => {
|
||||
if(!brokerToLeaderAndISRInfosMap.contains(b))
|
||||
brokerToLeaderAndISRInfosMap.put(b, new mutable.HashMap[(String, Int), LeaderAndISR])
|
||||
brokerToLeaderAndISRInfosMap(b).put(topicPartition, leaderAndISR)
|
||||
})
|
||||
allLeaders.put(topicPartition, leaderAndISR.leader)
|
||||
}
|
||||
} else
|
||||
debug("during leaderAndISR recovery, the leader %d is not alive any more, just ignore it".format(leaderAndISR.leader))
|
||||
}
|
||||
info("during leaderAndISR recovery, the broker to request map is [%s]".format(brokerToLeaderAndISRInfosMap.toString()))
|
||||
|
||||
brokerToLeaderAndISRInfosMap.foreach(m =>{
|
||||
val broker = m._1
|
||||
val leaderAndISRs = m._2
|
||||
val leaderAndISRRequest = new LeaderAndISRRequest(LeaderAndISRRequest.IsInit, leaderAndISRs)
|
||||
info("during leaderAndISR recovery, the leaderAndISRRequest sent to new broker [%s] is [%s]".format(broker, leaderAndISRRequest.toString))
|
||||
sendRequest(broker, leaderAndISRRequest)
|
||||
})
|
||||
|
||||
info("after leaderAndISR recovery for brokers %s, the leaders assignment is %s".format(brokerIds, allLeaders))
|
||||
}
|
||||
|
||||
|
||||
private def initLeaders(partitionReplicaAssignment: collection.mutable.Map[(String, Int), Seq[Int]]) {
|
||||
val brokerToLeaderAndISRInfosMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int),LeaderAndISR]]
|
||||
for((topicPartition, replicaAssignment) <- partitionReplicaAssignment) {
|
||||
val liveAssignedReplicas = replicaAssignment.filter(r => allBrokerIds.contains(r))
|
||||
debug("for topic [%s], partition [%d], live assigned replicas are: [%s]"
|
||||
.format(topicPartition._1,
|
||||
topicPartition._2,
|
||||
liveAssignedReplicas))
|
||||
if(!liveAssignedReplicas.isEmpty){
|
||||
debug("live assigned replica is not empty, check zkClient: %s".format(zkClient))
|
||||
val leader = liveAssignedReplicas.head
|
||||
var leaderAndISR: LeaderAndISR = null
|
||||
var updateLeaderISRZKPathSucceeded: Boolean = false
|
||||
while(!updateLeaderISRZKPathSucceeded){
|
||||
val curLeaderAndISROpt = ZkUtils.getLeaderAndISRForPartition(zkClient, topicPartition._1, topicPartition._2)
|
||||
debug("curLeaderAndISROpt is %s, zkClient is %s ".format(curLeaderAndISROpt, zkClient))
|
||||
if(curLeaderAndISROpt == None){
|
||||
debug("during initializing leader of parition (%s, %d), the current leader and isr in zookeeper is empty".format(topicPartition._1, topicPartition._2))
|
||||
leaderAndISR = new LeaderAndISR(leader, liveAssignedReplicas.toList)
|
||||
ZkUtils.createPersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndISRPath(topicPartition._1, topicPartition._2), leaderAndISR.toString)
|
||||
updateLeaderISRZKPathSucceeded = true
|
||||
} else{
|
||||
debug("during initializing leader of parition (%s, %d), the current leader and isr in zookeeper is not empty".format(topicPartition._1, topicPartition._2))
|
||||
val curZkPathVersion = curLeaderAndISROpt.get.zkVersion
|
||||
leaderAndISR = new LeaderAndISR(leader, curLeaderAndISROpt.get.leaderEpoch + 1,liveAssignedReplicas.toList, curLeaderAndISROpt.get.zkVersion + 1)
|
||||
val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndISRPath(topicPartition._1, topicPartition._2), leaderAndISR.toString, curZkPathVersion)
|
||||
if(updateSucceeded){
|
||||
leaderAndISR.zkVersion = newVersion
|
||||
}
|
||||
updateLeaderISRZKPathSucceeded = updateSucceeded
|
||||
}
|
||||
}
|
||||
liveAssignedReplicas.foreach(b => {
|
||||
if(!brokerToLeaderAndISRInfosMap.contains(b))
|
||||
brokerToLeaderAndISRInfosMap.put(b, new mutable.HashMap[(String, Int), LeaderAndISR])
|
||||
brokerToLeaderAndISRInfosMap(b).put(topicPartition, leaderAndISR)
|
||||
}
|
||||
)
|
||||
allLeaders.put(topicPartition, leaderAndISR.leader)
|
||||
}
|
||||
else{
|
||||
warn("during initializing leader of parition (%s, %d), assigned replicas are [%s], live brokers are [%s], no assigned replica is alive".format(topicPartition._1, topicPartition._2, replicaAssignment, allBrokerIds))
|
||||
}
|
||||
}
|
||||
|
||||
info("after leaders initialization for partition replica assignments %s, the cached leaders in controller is %s, and the broker to request map is: %s".format(partitionReplicaAssignment, allLeaders, brokerToLeaderAndISRInfosMap))
|
||||
brokerToLeaderAndISRInfosMap.foreach(m =>{
|
||||
val broker = m._1
|
||||
val leaderAndISRs = m._2
|
||||
val leaderAndISRRequest = new LeaderAndISRRequest(LeaderAndISRRequest.NotInit, leaderAndISRs)
|
||||
info("at initializing leaders for new partitions, the leaderAndISR request sent to broker %d is %s".format(broker, leaderAndISRRequest))
|
||||
sendRequest(broker, leaderAndISRRequest)
|
||||
})
|
||||
}
|
||||
|
||||
|
||||
private def onBrokerChange(newBrokers: Set[Int] = null){
|
||||
/** handle the new brokers, send request for them to initialize the local log **/
|
||||
if(newBrokers != null)
|
||||
deliverLeaderAndISRFromZookeeper(newBrokers, allTopics)
|
||||
|
||||
/** handle leader election for the partitions whose leader is no longer alive **/
|
||||
val brokerToLeaderAndISRInfosMap = new mutable.HashMap[Int, mutable.HashMap[(String, Int), LeaderAndISR]]
|
||||
allLeaders.foreach(m =>{
|
||||
val topicPartition = m._1
|
||||
val leader = m._2
|
||||
// We only care about the partitions, whose leader is no longer alive
|
||||
if(!allBrokerIds.contains(leader)){
|
||||
var updateLeaderISRZKPathSucceeded: Boolean = false
|
||||
while(!updateLeaderISRZKPathSucceeded){
|
||||
val assignedReplicasOpt = allPartitionReplicaAssignment.get(topicPartition)
|
||||
if(assignedReplicasOpt == None)
|
||||
throw new IllegalStateException("On broker changes, the assigned replica for [%s, %d], shouldn't be None, the general assignment is %s".format(topicPartition._1, topicPartition._2, allPartitionReplicaAssignment))
|
||||
val assignedReplicas = assignedReplicasOpt.get
|
||||
val liveAssignedReplicasToThisPartition = assignedReplicas.filter(r => allBrokerIds.contains(r))
|
||||
val curLeaderAndISROpt = ZkUtils.getLeaderAndISRForPartition(zkClient, topicPartition._1, topicPartition._2)
|
||||
if(curLeaderAndISROpt == None){
|
||||
throw new IllegalStateException("On broker change, there's no leaderAndISR information for partition (%s, %d) in zookeeper".format(topicPartition._1, topicPartition._2))
|
||||
}
|
||||
val curLeaderAndISR = curLeaderAndISROpt.get
|
||||
val leader = curLeaderAndISR.leader
|
||||
var newLeader: Int = -1
|
||||
val leaderEpoch = curLeaderAndISR.leaderEpoch
|
||||
val ISR = curLeaderAndISR.ISR
|
||||
val curZkPathVersion = curLeaderAndISR.zkVersion
|
||||
debug("leader, epoch, ISR and zkPathVersion for partition (%s, %d) are: [%d], [%d], [%s], [%d]".format(topicPartition._1, topicPartition._2, leader, leaderEpoch, ISR, curZkPathVersion))
|
||||
// The leader is no longer alive, need reelection, we only care about the leader change here, the ISR change can be handled by the leader
|
||||
var leaderAndISR: LeaderAndISR = null
|
||||
// The ISR contains at least 1 broker in the live broker list
|
||||
val liveBrokersInISR = ISR.filter(r => allBrokerIds.contains(r))
|
||||
if(!liveBrokersInISR.isEmpty){
|
||||
newLeader = liveBrokersInISR.head
|
||||
leaderAndISR = new LeaderAndISR(newLeader, leaderEpoch +1, liveBrokersInISR.toList, curZkPathVersion + 1)
|
||||
debug("some broker in ISR is alive, new leader and ISR is %s".format(leaderAndISR.toString()))
|
||||
} else{
|
||||
debug("live broker in ISR is empty, see live assigned replicas: %s".format(liveAssignedReplicasToThisPartition))
|
||||
if (!liveAssignedReplicasToThisPartition.isEmpty){
|
||||
newLeader = liveAssignedReplicasToThisPartition.head
|
||||
leaderAndISR = new LeaderAndISR(newLeader, leaderEpoch + 1, List(newLeader), curZkPathVersion + 1)
|
||||
warn("on broker change, no broker in ISR is alive, new leader elected is [%s], there's potential data loss".format(newLeader))
|
||||
} else
|
||||
error("on broker change, for partition ([%s, %d]), live brokers are: [%s], assigned replicas are: [%s]; no asigned replica is alive".format(topicPartition._1, topicPartition._2, allBrokerIds, assignedReplicas))
|
||||
}
|
||||
debug("the leader and ISR converted string: [%s]".format(leaderAndISR))
|
||||
val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndISRPath(topicPartition._1, topicPartition._2), leaderAndISR.toString, curZkPathVersion)
|
||||
if(updateSucceeded){
|
||||
leaderAndISR.zkVersion = newVersion
|
||||
liveAssignedReplicasToThisPartition.foreach(b => {
|
||||
if(!brokerToLeaderAndISRInfosMap.contains(b))
|
||||
brokerToLeaderAndISRInfosMap.put(b, new mutable.HashMap[(String, Int), LeaderAndISR])
|
||||
brokerToLeaderAndISRInfosMap(b).put(topicPartition, leaderAndISR)
|
||||
})
|
||||
allLeaders.put(topicPartition, newLeader)
|
||||
info("on broker changes, allLeader is updated to %s".format(allLeaders))
|
||||
}
|
||||
updateLeaderISRZKPathSucceeded = updateSucceeded
|
||||
}
|
||||
}
|
||||
})
|
||||
trace("after acting on broker change, the broker to leaderAndISR request map is".format(brokerToLeaderAndISRInfosMap))
|
||||
brokerToLeaderAndISRInfosMap.foreach(m => {
|
||||
val broker = m._1
|
||||
val leaderAndISRInfos = m._2
|
||||
val leaderAndISRRequest = new LeaderAndISRRequest(LeaderAndISRRequest.NotInit, leaderAndISRInfos)
|
||||
sendRequest(broker, leaderAndISRRequest)
|
||||
info("on broker change, the LeaderAndISRRequest send to brokers [%d] is [%s]".format(leaderAndISRRequest, broker))
|
||||
})
|
||||
}
|
||||
|
||||
class BrokerChangeListener() extends IZkChildListener with Logging {
|
||||
this.logIdent = "Controller " + config.brokerId + ", "
|
||||
def handleChildChange(parentPath : String, javaCurChildren : java.util.List[String]) {
|
||||
import scala.collection.JavaConversions._
|
||||
lock synchronized {
|
||||
info("Broker change listener at controller triggerred")
|
||||
val allBrokerIds = allBrokers.map(_.id)
|
||||
controllerLock synchronized {
|
||||
info("broker change listener triggered")
|
||||
val curChildrenSeq: Seq[String] = javaCurChildren
|
||||
val curBrokerIdsSeq = curChildrenSeq.map(_.toInt)
|
||||
val curBrokerIds = curBrokerIdsSeq.toSet
|
||||
val addedBrokerIds = curBrokerIds -- allBrokerIds
|
||||
val addedBrokersSeq = ZkUtils.getBrokerInfoFromIds(zkClient, addedBrokerIds.toSeq)
|
||||
info("Added brokers: " + addedBrokerIds.toString())
|
||||
val deletedBrokerIds = allBrokerIds -- curBrokerIds
|
||||
info("Deleted brokers: " + deletedBrokerIds.toString())
|
||||
|
||||
allBrokers = ZkUtils.getBrokerInfoFromIds(zkClient, curBrokerIdsSeq).toSet
|
||||
|
||||
for(broker <- addedBrokersSeq){
|
||||
controllerChannelManager.addBroker(broker)
|
||||
}
|
||||
for (brokerId <- deletedBrokerIds){
|
||||
controllerChannelManager.removeBroker(brokerId)
|
||||
}
|
||||
/** TODO: add other broker change handler logic**/
|
||||
allBrokerIds = allBrokers.map(_.id)
|
||||
info("added brokers: %s, deleted brokers: %s, all brokers: %s".format(addedBrokerIds, deletedBrokerIds, allBrokerIds))
|
||||
addedBrokersSeq.foreach(controllerChannelManager.addBroker(_))
|
||||
deletedBrokerIds.foreach(controllerChannelManager.removeBroker(_))
|
||||
onBrokerChange(addedBrokerIds)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def handleNewTopics(topics: Set[String], partitionReplicaAssignment: mutable.Map[(String, Int), Seq[Int]]) {
|
||||
// get relevant partitions to this broker
|
||||
val partitionReplicaAssignment = allPartitionReplicaAssignment.filter(p => topics.contains(p._1._1))
|
||||
trace("handling new topics, the partition replica assignment to be handled is %s".format(partitionReplicaAssignment))
|
||||
initLeaders(partitionReplicaAssignment)
|
||||
}
|
||||
|
||||
private def handleDeletedTopics(topics: Set[String], partitionReplicaAssignment: mutable.Map[(String, Int), Seq[Int]]) {
|
||||
val brokerToPartitionToStopReplicaMap = new collection.mutable.HashMap[Int, collection.mutable.HashSet[(String, Int)]]
|
||||
for((topicPartition, brokers) <- partitionReplicaAssignment){
|
||||
for (broker <- brokers){
|
||||
if (!brokerToPartitionToStopReplicaMap.contains(broker))
|
||||
brokerToPartitionToStopReplicaMap.put(broker, new collection.mutable.HashSet[(String, Int)])
|
||||
brokerToPartitionToStopReplicaMap(broker).add(topicPartition)
|
||||
}
|
||||
allLeaders.remove(topicPartition)
|
||||
info("after deleting topics %s, allLeader is updated to %s and the broker to stop replia request map is %s".format(topics, allLeaders, brokerToPartitionToStopReplicaMap))
|
||||
ZkUtils.deletePath(zkClient, ZkUtils.getTopicPartitionLeaderAndISRPath(topicPartition._1, topicPartition._2))
|
||||
}
|
||||
for((broker, partitionToStopReplica) <- brokerToPartitionToStopReplicaMap){
|
||||
val stopReplicaRequest = new StopReplicaRequest(partitionToStopReplica)
|
||||
info("handling deleted topics: [%s] the stopReplicaRequest sent to broker %d is [%s]".format(topics, broker, stopReplicaRequest))
|
||||
sendRequest(broker, stopReplicaRequest)
|
||||
}
|
||||
/*TODO: kafka-330 remove the unneeded leaderAndISRPath that the previous controller didn't get a chance to remove*/
|
||||
}
|
||||
|
||||
class TopicChangeListener extends IZkChildListener with Logging {
|
||||
this.logIdent = "Controller " + config.brokerId + ", "
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleChildChange(parentPath : String, curChilds : java.util.List[String]) {
|
||||
// TODO: Incomplete, do not need to review this time
|
||||
controllerLock synchronized {
|
||||
info("topic/partition change listener fired for path " + parentPath)
|
||||
val currentChildren = JavaConversions.asBuffer(curChilds).toSet
|
||||
val newTopics = currentChildren -- allTopics
|
||||
val deletedTopics = allTopics -- currentChildren
|
||||
val deletedPartitionReplicaAssignment = allPartitionReplicaAssignment.filter(p => deletedTopics.contains(p._1._1))
|
||||
allTopics = currentChildren
|
||||
|
||||
val addedPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, newTopics.iterator)
|
||||
allPartitionReplicaAssignment = allPartitionReplicaAssignment.filter(p => !deletedTopics.contains(p._1._1))
|
||||
allPartitionReplicaAssignment.++=(addedPartitionReplicaAssignment)
|
||||
info("new topics: [%s], deleted topics: [%s], new partition replica assignment [%s]".format(newTopics, deletedTopics, allPartitionReplicaAssignment))
|
||||
handleNewTopics(newTopics, addedPartitionReplicaAssignment)
|
||||
handleDeletedTopics(deletedTopics, deletedPartitionReplicaAssignment)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ControllerExistListener extends IZkDataListener with Logging {
|
||||
this.logIdent = "Controller " + config.brokerId + ", "
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleDataChange(dataPath: String, data: Object) {
|
||||
// do nothing, since No logic is needed here
|
||||
|
@ -285,8 +532,10 @@ class KafkaController(config : KafkaConfig) extends Logging {
|
|||
|
||||
@throws(classOf[Exception])
|
||||
def handleDataDeleted(dataPath: String) {
|
||||
info("Controller fail over, broker " + config.brokerId + " try to become controller")
|
||||
tryToBecomeController()
|
||||
controllerLock synchronized {
|
||||
info("the current controller failed, competes to be new controller")
|
||||
controllerRegisterOrFailover()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -24,41 +24,44 @@ import java.util.concurrent.atomic.AtomicLong
|
|||
/**
|
||||
* A thread that answers kafka requests.
|
||||
*/
|
||||
class KafkaRequestHandler(val requestChannel: RequestChannel, apis: KafkaApis) extends Runnable with Logging {
|
||||
|
||||
class KafkaRequestHandler(id: Int, brokerId: Int, val requestChannel: RequestChannel, apis: KafkaApis) extends Runnable with Logging {
|
||||
this.logIdent = "Kafka Request Handler " + id + " on Broker " + brokerId + ", "
|
||||
|
||||
def run() {
|
||||
while(true) {
|
||||
val req = requestChannel.receiveRequest()
|
||||
trace("Processor " + Thread.currentThread.getName + " got request " + req)
|
||||
if(req == RequestChannel.AllDone)
|
||||
if(req == RequestChannel.AllDone){
|
||||
trace("receives shut down command, shut down".format(brokerId, id))
|
||||
return
|
||||
}
|
||||
debug("handles request " + req)
|
||||
apis.handle(req)
|
||||
}
|
||||
}
|
||||
|
||||
def shutdown(): Unit = requestChannel.sendRequest(RequestChannel.AllDone)
|
||||
|
||||
}
|
||||
|
||||
class KafkaRequestHandlerPool(val requestChannel: RequestChannel,
|
||||
val apis: KafkaApis,
|
||||
class KafkaRequestHandlerPool(val brokerId: Int,
|
||||
val requestChannel: RequestChannel,
|
||||
val apis: KafkaApis,
|
||||
numThreads: Int) extends Logging {
|
||||
|
||||
this.logIdent = "Kafka Request Handler on Broker " + brokerId + ", "
|
||||
val threads = new Array[Thread](numThreads)
|
||||
val runnables = new Array[KafkaRequestHandler](numThreads)
|
||||
for(i <- 0 until numThreads) {
|
||||
runnables(i) = new KafkaRequestHandler(requestChannel, apis)
|
||||
runnables(i) = new KafkaRequestHandler(i, brokerId, requestChannel, apis)
|
||||
threads(i) = Utils.daemonThread("kafka-request-handler-" + i, runnables(i))
|
||||
threads(i).start()
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
info("Shutting down request handlers")
|
||||
info("shutting down")
|
||||
for(handler <- runnables)
|
||||
handler.shutdown
|
||||
for(thread <- threads)
|
||||
thread.join
|
||||
info("Request handlers shut down")
|
||||
info("shutted down completely")
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -24,8 +24,9 @@ import kafka.utils._
|
|||
import java.util.concurrent._
|
||||
import atomic.AtomicBoolean
|
||||
import kafka.cluster.Replica
|
||||
import kafka.api.LeaderAndISR
|
||||
import scala.collection._
|
||||
import org.I0Itec.zkclient.ZkClient
|
||||
import kafka.common.KafkaZookeeperClient
|
||||
|
||||
|
||||
/**
|
||||
|
@ -33,7 +34,7 @@ import kafka.common.KafkaZookeeperClient
|
|||
* to start up and shutdown a single Kafka node.
|
||||
*/
|
||||
class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logging {
|
||||
|
||||
this.logIdent = "Kafka Server " + config.brokerId + ", "
|
||||
val CleanShutdownFile = ".kafka_cleanshutdown"
|
||||
private var isShuttingDown = new AtomicBoolean(false)
|
||||
private var shutdownLatch = new CountDownLatch(1)
|
||||
|
@ -44,7 +45,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
|||
var kafkaZookeeper: KafkaZooKeeper = null
|
||||
var replicaManager: ReplicaManager = null
|
||||
private var apis: KafkaApis = null
|
||||
var kafkaController: KafkaController = new KafkaController(config)
|
||||
var kafkaController: KafkaController = null
|
||||
val kafkaScheduler = new KafkaScheduler(4)
|
||||
var zkClient: ZkClient = null
|
||||
|
||||
|
@ -53,7 +54,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
|||
* Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers
|
||||
*/
|
||||
def startup() {
|
||||
info("Starting Kafka server..." + config.brokerId)
|
||||
info("starting")
|
||||
isShuttingDown = new AtomicBoolean(false)
|
||||
shutdownLatch = new CountDownLatch(1)
|
||||
var needRecovery = true
|
||||
|
@ -62,11 +63,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
|||
needRecovery = false
|
||||
cleanShutDownFile.delete
|
||||
}
|
||||
/* start client */
|
||||
info("Connecting to ZK: " + config.zkConnect)
|
||||
zkClient = KafkaZookeeperClient.getZookeeperClient(config)
|
||||
|
||||
/* start scheduler */
|
||||
kafkaScheduler.startUp
|
||||
|
||||
/* start log manager */
|
||||
logManager = new LogManager(config,
|
||||
kafkaScheduler,
|
||||
|
@ -75,88 +75,107 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
|||
1000L * 60 * 60 * config.logRetentionHours,
|
||||
needRecovery)
|
||||
logManager.startup()
|
||||
|
||||
socketServer = new SocketServer(config.port,
|
||||
|
||||
socketServer = new SocketServer(config.brokerId,
|
||||
config.port,
|
||||
config.numNetworkThreads,
|
||||
config.monitoringPeriodSecs,
|
||||
config.numQueuedRequests,
|
||||
config.maxSocketRequestSize)
|
||||
|
||||
socketServer.startup
|
||||
|
||||
Utils.registerMBean(socketServer.stats, statsMBeanName)
|
||||
|
||||
kafkaZookeeper = new KafkaZooKeeper(config, zkClient, addReplica, getReplica, makeLeader, makeFollower)
|
||||
/* start client */
|
||||
kafkaZookeeper = new KafkaZooKeeper(config)
|
||||
// starting relevant replicas and leader election for partitions assigned to this broker
|
||||
kafkaZookeeper.startup
|
||||
|
||||
replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler)
|
||||
info("Connecting to ZK: " + config.zkConnect)
|
||||
|
||||
apis = new KafkaApis(socketServer.requestChannel, logManager, replicaManager, kafkaZookeeper)
|
||||
requestHandlerPool = new KafkaRequestHandlerPool(socketServer.requestChannel, apis, config.numIoThreads)
|
||||
socketServer.startup()
|
||||
replicaManager = new ReplicaManager(config, time, kafkaZookeeper.getZookeeperClient, kafkaScheduler, deleteLog)
|
||||
|
||||
kafkaController = new KafkaController(config, kafkaZookeeper.getZookeeperClient)
|
||||
apis = new KafkaApis(socketServer.requestChannel, logManager, replicaManager, kafkaZookeeper,
|
||||
addReplica, stopReplica, makeLeader, makeFollower, config.brokerId)
|
||||
requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads)
|
||||
Mx4jLoader.maybeLoad
|
||||
|
||||
// starting relevant replicas and leader election for partitions assigned to this broker
|
||||
kafkaZookeeper.startup()
|
||||
/**
|
||||
* Registers this broker in ZK. After this, consumers can connect to broker.
|
||||
* So this should happen after socket server start.
|
||||
*/
|
||||
// start the replica manager
|
||||
replicaManager.startup()
|
||||
// start the controller
|
||||
kafkaController.startup()
|
||||
|
||||
info("Server started.")
|
||||
info("started")
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Shutdown API for shutting down a single instance of the Kafka server.
|
||||
* Shuts down the LogManager, the SocketServer and the log cleaner scheduler thread
|
||||
*/
|
||||
def shutdown() {
|
||||
info("shutting down")
|
||||
val canShutdown = isShuttingDown.compareAndSet(false, true);
|
||||
if (canShutdown) {
|
||||
info("Shutting down Kafka server with id " + config.brokerId)
|
||||
if(requestHandlerPool != null)
|
||||
requestHandlerPool.shutdown()
|
||||
kafkaScheduler.shutdown()
|
||||
apis.close()
|
||||
kafkaZookeeper.shutdown()
|
||||
if(replicaManager != null)
|
||||
replicaManager.shutdown()
|
||||
if (socketServer != null)
|
||||
socketServer.shutdown()
|
||||
if(requestHandlerPool != null)
|
||||
requestHandlerPool.shutdown()
|
||||
Utils.unregisterMBean(statsMBeanName)
|
||||
if(logManager != null)
|
||||
logManager.shutdown()
|
||||
|
||||
if(kafkaController != null)
|
||||
kafkaController.shutDown()
|
||||
kafkaZookeeper.shutdown()
|
||||
zkClient.close()
|
||||
|
||||
val cleanShutDownFile = new File(new File(config.logDir), CleanShutdownFile)
|
||||
debug("Creating clean shutdown file " + cleanShutDownFile.getAbsolutePath())
|
||||
debug("creating clean shutdown file " + cleanShutDownFile.getAbsolutePath())
|
||||
cleanShutDownFile.createNewFile
|
||||
shutdownLatch.countDown()
|
||||
info("Kafka server with id %d shut down completed".format(config.brokerId))
|
||||
info("shutted down completed")
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* After calling shutdown(), use this API to wait until the shutdown is complete
|
||||
*/
|
||||
def awaitShutdown(): Unit = shutdownLatch.await()
|
||||
|
||||
def addReplica(topic: String, partition: Int, assignedReplicas: Set[Int]): Replica = {
|
||||
info("Added local replica for topic %s partition %d on broker %d".format(topic, partition, config.brokerId))
|
||||
// get local log
|
||||
val log = logManager.getOrCreateLog(topic, partition)
|
||||
replicaManager.addLocalReplica(topic, partition, log, assignedReplicas)
|
||||
}
|
||||
|
||||
def makeLeader(replica: Replica, currentISRInZk: Seq[Int]) {
|
||||
replicaManager.makeLeader(replica, currentISRInZk)
|
||||
def makeLeader(replica: Replica, leaderAndISR: LeaderAndISR): Short = {
|
||||
replicaManager.makeLeader(replica, leaderAndISR)
|
||||
}
|
||||
|
||||
def makeFollower(replica: Replica, leaderBrokerId: Int, zkClient: ZkClient) {
|
||||
replicaManager.makeFollower(replica, leaderBrokerId, zkClient)
|
||||
def makeFollower(replica: Replica, leaderAndISR: LeaderAndISR): Short = {
|
||||
replicaManager.makeFollower(replica, leaderAndISR)
|
||||
}
|
||||
|
||||
def getReplica(topic: String, partition: Int): Option[Replica] =
|
||||
replicaManager.getReplica(topic, partition)
|
||||
|
||||
def stopReplica(topic: String, partition: Int): Short = {
|
||||
replicaManager.stopReplica(topic, partition)
|
||||
}
|
||||
|
||||
def deleteLog(topic: String, partition: Int): Unit = {
|
||||
/* TODO: handle deleteLog in a better way */
|
||||
//logManager.deleteLog(topic, partition)
|
||||
}
|
||||
|
||||
def getLogManager(): LogManager = logManager
|
||||
|
||||
def getStats(): SocketServerStats = socketServer.stats
|
||||
|
|
|
@ -18,46 +18,29 @@
|
|||
package kafka.server
|
||||
|
||||
import java.net.InetAddress
|
||||
import kafka.cluster.Replica
|
||||
import kafka.utils._
|
||||
import org.apache.zookeeper.Watcher.Event.KeeperState
|
||||
import org.I0Itec.zkclient.{IZkDataListener, IZkChildListener, IZkStateListener, ZkClient}
|
||||
import kafka.admin.AdminUtils
|
||||
import java.lang.Thread
|
||||
import collection.mutable.HashSet
|
||||
import org.I0Itec.zkclient.{IZkStateListener, ZkClient}
|
||||
import kafka.common._
|
||||
|
||||
|
||||
/**
|
||||
* Handles the server's interaction with zookeeper. The server needs to register the following paths:
|
||||
* /topics/[topic]/[node_id-partition_num]
|
||||
* /brokers/[0...N] --> host:port
|
||||
*
|
||||
*/
|
||||
class KafkaZooKeeper(config: KafkaConfig,
|
||||
zkClient: ZkClient,
|
||||
addReplicaCbk: (String, Int, Set[Int]) => Replica,
|
||||
getReplicaCbk: (String, Int) => Option[Replica],
|
||||
becomeLeader: (Replica, Seq[Int]) => Unit,
|
||||
becomeFollower: (Replica, Int, ZkClient) => Unit) extends Logging {
|
||||
class KafkaZooKeeper(config: KafkaConfig) extends Logging {
|
||||
|
||||
val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + config.brokerId
|
||||
private var leaderChangeListener: LeaderChangeListener = null
|
||||
private var topicPartitionsChangeListener: TopicChangeListener = null
|
||||
private var stateChangeHandler: StateChangeCommandHandler = null
|
||||
private var zkClient: ZkClient = null
|
||||
|
||||
private val topicListenerLock = new Object
|
||||
private val leaderChangeLock = new Object
|
||||
|
||||
def startup() {
|
||||
leaderChangeListener = new LeaderChangeListener
|
||||
topicPartitionsChangeListener = new TopicChangeListener
|
||||
leaderChangeListener = new LeaderChangeListener
|
||||
topicPartitionsChangeListener = new TopicChangeListener
|
||||
startStateChangeCommandHandler()
|
||||
zkClient.subscribeStateChanges(new SessionExpireListener)
|
||||
registerBrokerInZk()
|
||||
subscribeToTopicAndPartitionsChanges(true)
|
||||
}
|
||||
def startup() {
|
||||
/* start client */
|
||||
info("connecting to ZK: " + config.zkConnect)
|
||||
zkClient = KafkaZookeeperClient.getZookeeperClient(config)
|
||||
zkClient.subscribeStateChanges(new SessionExpireListener)
|
||||
registerBrokerInZk()
|
||||
}
|
||||
|
||||
private def registerBrokerInZk() {
|
||||
info("Registering broker " + brokerIdPath)
|
||||
|
@ -66,13 +49,6 @@ class KafkaZooKeeper(config: KafkaConfig,
|
|||
ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, creatorId, config.port)
|
||||
}
|
||||
|
||||
private def startStateChangeCommandHandler() {
|
||||
val stateChangeQ = new ZkQueue(zkClient, ZkUtils.getBrokerStateChangePath(config.brokerId), config.stateChangeQSize)
|
||||
stateChangeHandler = new StateChangeCommandHandler("StateChangeCommandHandler", config, stateChangeQ,
|
||||
ensureStateChangeCommandValidityOnThisBroker, ensureEpochValidity)
|
||||
stateChangeHandler.start()
|
||||
}
|
||||
|
||||
/**
|
||||
* When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a
|
||||
* connection for us. We need to re-register this broker in the broker registry.
|
||||
|
@ -96,20 +72,24 @@ class KafkaZooKeeper(config: KafkaConfig,
|
|||
registerBrokerInZk()
|
||||
info("done re-registering broker")
|
||||
info("Subscribing to %s path to watch for new topics".format(ZkUtils.BrokerTopicsPath))
|
||||
zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicPartitionsChangeListener)
|
||||
val topics = ZkUtils.getAllTopics(zkClient)
|
||||
debug("Existing topics are %s".format(topics.mkString(",")))
|
||||
topics.foreach(topic => zkClient.subscribeChildChanges(ZkUtils.getTopicPartitionsPath(topic), topicPartitionsChangeListener))
|
||||
handleNewTopics(topics)
|
||||
}
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
stateChangeHandler.shutdown()
|
||||
if (zkClient != null) {
|
||||
info("Closing zookeeper client...")
|
||||
zkClient.close()
|
||||
}
|
||||
}
|
||||
|
||||
private def doesTopicExistInCluster(topic: String) : Boolean = {
|
||||
val allTopics = ZkUtils.getAllTopics(zkClient)
|
||||
trace("all topics, %s, topic %s".format(allTopics, topic))
|
||||
allTopics.contains(topic)
|
||||
}
|
||||
|
||||
def ensurePartitionLeaderOnThisBroker(topic: String, partition: Int) {
|
||||
if(!topicPartitionsChangeListener.doesTopicExistInCluster(topic))
|
||||
if(!doesTopicExistInCluster(topic))
|
||||
throw new UnknownTopicException("Topic %s doesn't exist in the cluster".format(topic))
|
||||
// check if partition id is invalid
|
||||
if(partition < 0)
|
||||
|
@ -124,256 +104,7 @@ class KafkaZooKeeper(config: KafkaConfig,
|
|||
}
|
||||
}
|
||||
|
||||
def getZookeeperClient = zkClient
|
||||
|
||||
def handleNewTopics(topics: Seq[String]) {
|
||||
// get relevant partitions to this broker
|
||||
val topicsAndPartitionsOnThisBroker = ZkUtils.getPartitionsAssignedToBroker(zkClient, topics, config.brokerId)
|
||||
debug("Partitions assigned to broker %d are %s".format(config.brokerId, topicsAndPartitionsOnThisBroker.mkString(",")))
|
||||
for( (topic, partitionsAssignedToThisBroker) <- topicsAndPartitionsOnThisBroker ) {
|
||||
// subscribe to leader changes for these partitions
|
||||
subscribeToLeaderForPartitions(topic, partitionsAssignedToThisBroker)
|
||||
// start replicas for these partitions
|
||||
startReplicasForPartitions(topic, partitionsAssignedToThisBroker)
|
||||
}
|
||||
}
|
||||
|
||||
def subscribeToTopicAndPartitionsChanges(startReplicas: Boolean) {
|
||||
info("Subscribing to %s path to watch for new topics".format(ZkUtils.BrokerTopicsPath))
|
||||
zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicPartitionsChangeListener)
|
||||
val topics = ZkUtils.getAllTopics(zkClient)
|
||||
val topicsAndPartitionsOnThisBroker = ZkUtils.getPartitionsAssignedToBroker(zkClient, topics, config.brokerId)
|
||||
debug("Partitions assigned to broker %d are %s".format(config.brokerId, topicsAndPartitionsOnThisBroker.mkString(",")))
|
||||
for( (topic, partitionsAssignedToThisBroker) <- topicsAndPartitionsOnThisBroker ) {
|
||||
// subscribe to leader changes for these partitions
|
||||
subscribeToLeaderForPartitions(topic, partitionsAssignedToThisBroker)
|
||||
|
||||
// start replicas for these partitions
|
||||
if(startReplicas)
|
||||
startReplicasForPartitions(topic, partitionsAssignedToThisBroker)
|
||||
}
|
||||
}
|
||||
|
||||
private def subscribeToLeaderForPartitions(topic: String, partitions: Seq[Int]) {
|
||||
partitions.foreach { partition =>
|
||||
info("Broker %d subscribing to leader changes for topic %s partition %d".format(config.brokerId, topic, partition))
|
||||
// register leader change listener
|
||||
zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderPath(topic, partition.toString), leaderChangeListener)
|
||||
}
|
||||
}
|
||||
|
||||
private def startReplicasForPartitions(topic: String, partitions: Seq[Int]) {
|
||||
partitions.foreach { partition =>
|
||||
val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partition).map(r => r.toInt)
|
||||
info("Assigned replicas list for topic %s partition %d is %s".format(topic, partition, assignedReplicas.mkString(",")))
|
||||
if(assignedReplicas.contains(config.brokerId)) {
|
||||
val replica = addReplicaCbk(topic, partition, assignedReplicas.toSet)
|
||||
startReplica(replica)
|
||||
} else
|
||||
warn("Ignoring partition %d of topic %s since broker %d doesn't host any replicas for it"
|
||||
.format(partition, topic, config.brokerId))
|
||||
}
|
||||
}
|
||||
|
||||
private def startReplica(replica: Replica) {
|
||||
info("Starting replica for topic %s partition %d on broker %d"
|
||||
.format(replica.topic, replica.partition.partitionId, replica.brokerId))
|
||||
ZkUtils.getLeaderForPartition(zkClient, replica.topic, replica.partition.partitionId) match {
|
||||
case Some(leader) =>
|
||||
info("Topic %s partition %d has leader %d".format(replica.topic, replica.partition.partitionId,leader))
|
||||
// check if this broker is the leader, if not, then become follower
|
||||
if(leader != config.brokerId)
|
||||
becomeFollower(replica, leader, zkClient)
|
||||
case None => // leader election
|
||||
leaderElection(replica)
|
||||
}
|
||||
}
|
||||
|
||||
def leaderElection(replica: Replica) {
|
||||
info("Broker %d electing leader for topic %s partition %d".format(config.brokerId, replica.topic, replica.partition.partitionId))
|
||||
// read the AR list for replica.partition from ZK
|
||||
val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, replica.topic, replica.partition.partitionId).map(_.toInt)
|
||||
val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, replica.topic, replica.partition.partitionId)
|
||||
val liveBrokers = ZkUtils.getSortedBrokerList(zkClient).map(_.toInt)
|
||||
if(canBecomeLeader(config.brokerId, replica.topic, replica.partition.partitionId, assignedReplicas, inSyncReplicas, liveBrokers)) {
|
||||
info("Broker %d will participate in leader election for topic %s partition %d"
|
||||
.format(config.brokerId, replica.topic, replica.partition.partitionId))
|
||||
// wait for some time if it is not the preferred replica
|
||||
try {
|
||||
if(replica.brokerId != assignedReplicas.head) {
|
||||
// sleep only if the preferred replica is alive
|
||||
if(liveBrokers.contains(assignedReplicas.head)) {
|
||||
info("Preferred replica %d for topic %s ".format(assignedReplicas.head, replica.topic) +
|
||||
"partition %d is alive. Waiting for %d ms to allow it to become leader"
|
||||
.format(replica.partition.partitionId, config.preferredReplicaWaitTime))
|
||||
Thread.sleep(config.preferredReplicaWaitTime)
|
||||
}
|
||||
}
|
||||
} catch {
|
||||
case e => // ignoring
|
||||
}
|
||||
val newLeaderEpochAndISR = ZkUtils.tryToBecomeLeaderForPartition(zkClient, replica.topic,
|
||||
replica.partition.partitionId, replica.brokerId)
|
||||
newLeaderEpochAndISR match {
|
||||
case Some(epochAndISR) =>
|
||||
info("Broker %d is leader for topic %s partition %d".format(replica.brokerId, replica.topic,
|
||||
replica.partition.partitionId))
|
||||
info("Current ISR for topic %s partition %d is %s".format(replica.topic, replica.partition.partitionId,
|
||||
epochAndISR._2.mkString(",")))
|
||||
becomeLeader(replica, epochAndISR._2)
|
||||
case None =>
|
||||
ZkUtils.getLeaderForPartition(zkClient, replica.topic, replica.partition.partitionId) match {
|
||||
case Some(leader) =>
|
||||
becomeFollower(replica, leader, zkClient)
|
||||
case None =>
|
||||
error("Lost leader for topic %s partition %d right after leader election".format(replica.topic,
|
||||
replica.partition.partitionId))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def canBecomeLeader(brokerId: Int, topic: String, partition: Int, assignedReplicas: Seq[Int],
|
||||
inSyncReplicas: Seq[Int], liveBrokers: Seq[Int]): Boolean = {
|
||||
// TODO: raise alert, mark the partition offline if no broker in the assigned replicas list is alive
|
||||
assert(assignedReplicas.size > 0, "There should be at least one replica in the assigned replicas list for topic " +
|
||||
" %s partition %d".format(topic, partition))
|
||||
inSyncReplicas.size > 0 match {
|
||||
case true => // check if this broker is in the ISR. If yes, return true
|
||||
inSyncReplicas.contains(brokerId) match {
|
||||
case true =>
|
||||
info("Broker %d can become leader since it is in the ISR %s".format(brokerId, inSyncReplicas.mkString(",")) +
|
||||
" for topic %s partition %d".format(topic, partition))
|
||||
true
|
||||
case false =>
|
||||
// check if any broker in the ISR is alive. If not, return true only if this broker is in the AR
|
||||
val liveBrokersInISR = inSyncReplicas.filter(r => liveBrokers.contains(r))
|
||||
liveBrokersInISR.isEmpty match {
|
||||
case true =>
|
||||
if(assignedReplicas.contains(brokerId)) {
|
||||
info("No broker in the ISR %s for topic %s".format(inSyncReplicas.mkString(","), topic) +
|
||||
" partition %d is alive. Broker %d can become leader since it is in the assigned replicas %s"
|
||||
.format(partition, brokerId, assignedReplicas.mkString(",")))
|
||||
true
|
||||
} else {
|
||||
info("No broker in the ISR %s for topic %s".format(inSyncReplicas.mkString(","), topic) +
|
||||
" partition %d is alive. Broker %d can become leader since it is in the assigned replicas %s"
|
||||
.format(partition, brokerId, assignedReplicas.mkString(",")))
|
||||
false
|
||||
}
|
||||
case false =>
|
||||
info("ISR for topic %s partition %d is %s. Out of these %s brokers are alive. Broker %d "
|
||||
.format(topic, partition, inSyncReplicas.mkString(",")) + "cannot become leader since it doesn't exist " +
|
||||
"in the ISR")
|
||||
false // let one of the live brokers in the ISR become the leader
|
||||
}
|
||||
}
|
||||
case false =>
|
||||
if(assignedReplicas.contains(brokerId)) {
|
||||
info("ISR for topic %s partition %d is empty. Broker %d can become leader since it "
|
||||
.format(topic, partition, brokerId) + "is part of the assigned replicas list")
|
||||
true
|
||||
} else {
|
||||
info("ISR for topic %s partition %d is empty. Broker %d cannot become leader since it "
|
||||
.format(topic, partition, brokerId) + "is not part of the assigned replicas list")
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class TopicChangeListener extends IZkChildListener with Logging {
|
||||
private val allTopics = new HashSet[String]()
|
||||
// read existing topics, if any
|
||||
allTopics ++= ZkUtils.getAllTopics(zkClient)
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleChildChange(parentPath : String, curChilds : java.util.List[String]) {
|
||||
import collection.JavaConversions
|
||||
topicListenerLock.synchronized {
|
||||
debug("Topic/partition change listener fired for path " + parentPath)
|
||||
val currentChildren = JavaConversions.asBuffer(curChilds).toSet
|
||||
val newTopics = currentChildren -- allTopics
|
||||
val deletedTopics = allTopics -- currentChildren
|
||||
allTopics.clear()
|
||||
allTopics ++= currentChildren
|
||||
|
||||
debug("New topics: [%s]. Deleted topics: [%s]".format(newTopics.mkString(","), deletedTopics.mkString(",")))
|
||||
debug("Current topics in the cluster: [%s]".format(allTopics.mkString(",")))
|
||||
handleNewTopics(newTopics.toSeq)
|
||||
// TODO: Handle topic deletions
|
||||
// handleDeletedTopics(deletedTopics.toSeq)
|
||||
}
|
||||
}
|
||||
|
||||
def doesTopicExistInCluster(topic: String): Boolean = {
|
||||
topicListenerLock.synchronized {
|
||||
allTopics.contains(topic)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def ensureStateChangeCommandValidityOnThisBroker(stateChangeCommand: StateChangeCommand): Boolean = {
|
||||
// check if this broker hosts a replica for this topic and partition
|
||||
ZkUtils.isPartitionOnBroker(zkClient, stateChangeCommand.topic, stateChangeCommand.partition, config.brokerId)
|
||||
}
|
||||
|
||||
private def ensureEpochValidity(stateChangeCommand: StateChangeCommand): Boolean = {
|
||||
// get the topic and partition that this request is meant for
|
||||
val topic = stateChangeCommand.topic
|
||||
val partition = stateChangeCommand.partition
|
||||
val epoch = stateChangeCommand.epoch
|
||||
|
||||
val currentLeaderEpoch = ZkUtils.getEpochForPartition(zkClient, topic, partition)
|
||||
// check if the request's epoch matches the current leader's epoch OR the admin command's epoch
|
||||
val validEpoch = (currentLeaderEpoch == epoch) || (epoch == AdminUtils.AdminEpoch)
|
||||
if(epoch > currentLeaderEpoch)
|
||||
throw new IllegalStateException(("Illegal epoch state. Request's epoch %d larger than registered epoch %d for " +
|
||||
"topic %s partition %d").format(epoch, currentLeaderEpoch, topic, partition))
|
||||
validEpoch
|
||||
}
|
||||
|
||||
class LeaderChangeListener extends IZkDataListener with Logging {
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleDataChange(dataPath: String, data: Object) {
|
||||
// handle leader change event for path
|
||||
val newLeaderAndEpochInfo: String = data.asInstanceOf[String]
|
||||
val newLeader = newLeaderAndEpochInfo.split(";").head.toInt
|
||||
val newEpoch = newLeaderAndEpochInfo.split(";").last.toInt
|
||||
debug("Leader change listener fired on broker %d for path %s. New leader is %d. New epoch is %d".format(config.brokerId,
|
||||
dataPath, newLeader, newEpoch))
|
||||
val topicPartitionInfo = dataPath.split("/")
|
||||
val topic = topicPartitionInfo.takeRight(4).head
|
||||
val partition = topicPartitionInfo.takeRight(2).head.toInt
|
||||
info("Updating leader change information in replica for topic %s partition %d".format(topic, partition))
|
||||
val replica = getReplicaCbk(topic, partition).getOrElse(null)
|
||||
assert(replica != null, "Replica for topic %s partition %d should exist on broker %d"
|
||||
.format(topic, partition, config.brokerId))
|
||||
replica.partition.leaderId(Some(newLeader))
|
||||
assert(getReplicaCbk(topic, partition).get.partition.leaderId().get == newLeader, "New leader should be set correctly")
|
||||
}
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleDataDeleted(dataPath: String) {
|
||||
leaderChangeLock.synchronized {
|
||||
// leader is deleted for topic partition
|
||||
val topic = dataPath.split("/").takeRight(4).head
|
||||
val partitionId = dataPath.split("/").takeRight(2).head.toInt
|
||||
debug("Leader deleted listener fired for topic %s partition %d on broker %d"
|
||||
.format(topic, partitionId, config.brokerId))
|
||||
val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionId).map(r => r.toInt)
|
||||
if(assignedReplicas.contains(config.brokerId)) {
|
||||
val replica = getReplicaCbk(topic, partitionId)
|
||||
replica match {
|
||||
case Some(r) => leaderElection(r)
|
||||
case None => error("No replica exists for topic %s partition %s on broker %d"
|
||||
.format(topic, partitionId, config.brokerId))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
def getZookeeperClient = {
|
||||
zkClient
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -20,10 +20,10 @@ package kafka.server
|
|||
import kafka.cluster.Broker
|
||||
|
||||
class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager)
|
||||
extends AbstractFetcherManager("ReplicaFetcherManager", brokerConfig.numReplicaFetchers) {
|
||||
extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId + ", ", brokerConfig.numReplicaFetchers) {
|
||||
|
||||
override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = {
|
||||
new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(sourceBroker.id, fetcherId), sourceBroker, brokerConfig, replicaMgr)
|
||||
new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d on broker %d, ".format(sourceBroker.id, fetcherId, brokerConfig.brokerId), sourceBroker, brokerConfig, replicaMgr)
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
|
|
|
@ -18,64 +18,88 @@ package kafka.server
|
|||
|
||||
import kafka.log.Log
|
||||
import kafka.cluster.{Partition, Replica}
|
||||
import collection.mutable
|
||||
import collection._
|
||||
import mutable.ListBuffer
|
||||
import org.I0Itec.zkclient.ZkClient
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import kafka.utils.{KafkaScheduler, ZkUtils, Time, Logging}
|
||||
import kafka.common.{KafkaException, InvalidPartitionException}
|
||||
import kafka.api.LeaderAndISR
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import kafka.common.{BrokerNotExistException, KafkaException, ErrorMapping, InvalidPartitionException}
|
||||
|
||||
class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, kafkaScheduler: KafkaScheduler)
|
||||
extends Logging {
|
||||
|
||||
private var allReplicas = new mutable.HashMap[(String, Int), Partition]()
|
||||
class ReplicaManager(val config: KafkaConfig, time: Time, val zkClient: ZkClient, kafkaScheduler: KafkaScheduler, deleteLocalLog: (String, Int) => Unit) extends Logging {
|
||||
|
||||
var allPartitions = new mutable.HashMap[(String, Int), Partition]()
|
||||
private var leaderReplicas = new ListBuffer[Partition]()
|
||||
private val leaderReplicaLock = new ReentrantLock()
|
||||
private val replicaFetcherManager = new ReplicaFetcherManager(config, this)
|
||||
this.logIdent = "Replica Manager on Broker " + config.brokerId + ", "
|
||||
|
||||
val hwCheckPointThreadStarted = new AtomicBoolean(false)
|
||||
private val highwaterMarkCheckpoint = new HighwaterMarkCheckpoint(config.logDir)
|
||||
info("Created highwatermark file %s on broker %d".format(highwaterMarkCheckpoint.name, config.brokerId))
|
||||
info("Created highwatermark file %s".format(highwaterMarkCheckpoint.name))
|
||||
|
||||
def startHighWaterMarksCheckPointThread() = {
|
||||
if(hwCheckPointThreadStarted.compareAndSet(false, true))
|
||||
kafkaScheduler.scheduleWithRate(checkpointHighwaterMarks, "highwatermark-checkpoint-thread", 0, config.defaultFlushIntervalMs)
|
||||
}
|
||||
|
||||
def startup() {
|
||||
// start the highwatermark checkpoint thread
|
||||
kafkaScheduler.scheduleWithRate(checkpointHighwaterMarks, "highwatermark-checkpoint-thread", 0,
|
||||
config.defaultFlushIntervalMs)
|
||||
// start ISR expiration thread
|
||||
kafkaScheduler.scheduleWithRate(maybeShrinkISR, "isr-expiration-thread-", 0, config.replicaMaxLagTimeMs)
|
||||
}
|
||||
|
||||
def addLocalReplica(topic: String, partitionId: Int, log: Log, assignedReplicaIds: Set[Int]): Replica = {
|
||||
val partition = getOrCreatePartition(topic, partitionId, assignedReplicaIds)
|
||||
val localReplica = new Replica(config.brokerId, partition, topic, time,
|
||||
Some(readCheckpointedHighWatermark(topic, partitionId)), Some(log))
|
||||
|
||||
var retReplica : Replica = null
|
||||
val replicaOpt = partition.getReplica(config.brokerId)
|
||||
replicaOpt match {
|
||||
case Some(replica) =>
|
||||
info("Changing remote replica %s into a local replica".format(replica.toString))
|
||||
info("changing remote replica %s into a local replica".format(replica.toString))
|
||||
replica.log match {
|
||||
case None =>
|
||||
replica.log = Some(log)
|
||||
case Some(log) => // nothing to do since log already exists
|
||||
}
|
||||
retReplica = replica
|
||||
case None =>
|
||||
val localReplica = new Replica(config.brokerId, partition, topic, time,
|
||||
Some(readCheckpointedHighWatermark(topic, partitionId)), Some(log))
|
||||
partition.addReplica(localReplica)
|
||||
info("adding local replica %d for topic %s partition %s on broker %d".format(localReplica.brokerId, localReplica.topic, localReplica.partition.partitionId, localReplica.brokerId))
|
||||
retReplica = localReplica
|
||||
}
|
||||
val assignedReplicas = assignedReplicaIds.map(partition.getReplica(_).get)
|
||||
partition.assignedReplicas(Some(assignedReplicas))
|
||||
// get the replica objects for the assigned replicas for this partition
|
||||
info("Added local replica %d for topic %s partition %s on broker %d"
|
||||
.format(localReplica.brokerId, localReplica.topic, localReplica.partition.partitionId, localReplica.brokerId))
|
||||
localReplica
|
||||
retReplica
|
||||
}
|
||||
|
||||
def stopReplica(topic: String, partition: Int): Short = {
|
||||
trace("handling stop replica for partition [%s, %d]".format(topic, partition))
|
||||
val errorCode = ErrorMapping.NoError
|
||||
val replica = getReplica(topic, partition)
|
||||
if(replica.isDefined){
|
||||
replicaFetcherManager.removeFetcher(topic, partition)
|
||||
deleteLocalLog(topic, partition)
|
||||
allPartitions.remove((topic, partition))
|
||||
info("after removing partition (%s, %d), the rest of allReplicas is: [%s]".format(topic, partition, allPartitions))
|
||||
}
|
||||
trace("finishes handling stop replica [%s, %d]".format(topic, partition))
|
||||
errorCode
|
||||
}
|
||||
|
||||
|
||||
def getOrCreatePartition(topic: String, partitionId: Int, assignedReplicaIds: Set[Int]): Partition = {
|
||||
val newPartition = allReplicas.contains((topic, partitionId))
|
||||
val newPartition = allPartitions.contains((topic, partitionId))
|
||||
newPartition match {
|
||||
case true => // partition exists, do nothing
|
||||
allReplicas.get((topic, partitionId)).get
|
||||
allPartitions.get((topic, partitionId)).get
|
||||
case false => // create remote replicas for each replica id in assignedReplicas
|
||||
val partition = new Partition(topic, partitionId, time)
|
||||
allReplicas += (topic, partitionId) -> partition
|
||||
allPartitions += (topic, partitionId) -> partition
|
||||
(assignedReplicaIds - config.brokerId).foreach(
|
||||
replicaId => addRemoteReplica(topic, partitionId, replicaId, partition))
|
||||
partition
|
||||
|
@ -83,12 +107,11 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
}
|
||||
|
||||
def ensurePartitionExists(topic: String, partitionId: Int): Partition = {
|
||||
val partitionOpt = allReplicas.get((topic, partitionId))
|
||||
val partitionOpt = allPartitions.get((topic, partitionId))
|
||||
partitionOpt match {
|
||||
case Some(partition) => partition
|
||||
case None =>
|
||||
throw new InvalidPartitionException("Partition for topic %s partition %d doesn't exist in replica manager on %d"
|
||||
.format(topic, partitionId, config.brokerId))
|
||||
throw new InvalidPartitionException("Partition for topic %s partition %d doesn't exist in replica manager on %d".format(topic, partitionId, config.brokerId))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -97,32 +120,34 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
|
||||
val replicaAdded = partition.addReplica(remoteReplica)
|
||||
if(replicaAdded)
|
||||
info("Added remote replica %d for topic %s partition %s on broker %d"
|
||||
.format(remoteReplica.brokerId, remoteReplica.topic, remoteReplica.partition.partitionId, config.brokerId))
|
||||
info("added remote replica %d for topic %s partition %s".format(remoteReplica.brokerId, remoteReplica.topic, remoteReplica.partition.partitionId))
|
||||
remoteReplica
|
||||
}
|
||||
|
||||
def getReplica(topic: String, partitionId: Int, replicaId: Int = config.brokerId): Option[Replica] = {
|
||||
val replicasOpt = allReplicas.get((topic, partitionId))
|
||||
replicasOpt match {
|
||||
case Some(replicas) =>
|
||||
replicas.getReplica(replicaId)
|
||||
val partitionOpt = allPartitions.get((topic, partitionId))
|
||||
partitionOpt match {
|
||||
case Some(partition) =>
|
||||
partition.getReplica(replicaId)
|
||||
case None =>
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
def getLeaderReplica(topic: String, partitionId: Int): Option[Replica] = {
|
||||
val replicasOpt = allReplicas.get((topic, partitionId))
|
||||
val replicasOpt = allPartitions.get((topic, partitionId))
|
||||
replicasOpt match {
|
||||
case Some(replicas) =>
|
||||
Some(replicas.leaderReplica())
|
||||
case None =>
|
||||
throw new KafkaException("Getting leader replica failed. Partition replica metadata for topic " +
|
||||
"%s partition %d doesn't exist in Replica manager on %d".format(topic, partitionId, config.brokerId))
|
||||
"%s partition %d doesn't exist in Replica manager on %d".format(topic, partitionId, config.brokerId))
|
||||
}
|
||||
}
|
||||
|
||||
def getPartition(topic: String, partitionId: Int): Option[Partition] =
|
||||
allPartitions.get((topic, partitionId))
|
||||
|
||||
private def updateReplicaLeo(replica: Replica, fetchOffset: Long) {
|
||||
// set the replica leo
|
||||
val partition = ensurePartitionExists(replica.topic, replica.partition.partitionId)
|
||||
|
@ -137,38 +162,41 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
val newHw = allLeos.min
|
||||
val oldHw = partition.leaderHW()
|
||||
if(newHw > oldHw) {
|
||||
debug("Updating leader HW for topic %s partition %d to %d".format(replica.topic, replica.partition.partitionId, newHw))
|
||||
partition.leaderHW(Some(newHw))
|
||||
}else
|
||||
debug("Old hw for topic %s partition %d is %d. New hw is %d. All leo's are %s".format(replica.topic,
|
||||
replica.partition.partitionId, oldHw, newHw, allLeos.mkString(",")))
|
||||
replica.partition.partitionId, oldHw, newHw, allLeos.mkString(",")))
|
||||
}
|
||||
|
||||
def makeLeader(replica: Replica, currentISRInZk: Seq[Int]) {
|
||||
info("Broker %d started the leader state transition for topic %s partition %d"
|
||||
.format(config.brokerId, replica.topic, replica.partition.partitionId))
|
||||
def makeLeader(replica: Replica, leaderAndISR: LeaderAndISR): Short = {
|
||||
info("becoming Leader for topic [%s] partition [%d]".format(replica.topic, replica.partition.partitionId))
|
||||
info("started the leader state transition for topic %s partition %d"
|
||||
.format(replica.topic, replica.partition.partitionId))
|
||||
try {
|
||||
// read and cache the ISR
|
||||
replica.partition.leaderId(Some(replica.brokerId))
|
||||
replica.partition.updateISR(currentISRInZk.toSet)
|
||||
replica.partition.updateISR(leaderAndISR.ISR.toSet)
|
||||
// stop replica fetcher thread, if any
|
||||
replicaFetcherManager.removeFetcher(replica.topic, replica.partition.partitionId)
|
||||
// also add this partition to the list of partitions for which the leader is the current broker
|
||||
leaderReplicaLock.lock()
|
||||
leaderReplicas += replica.partition
|
||||
info("Broker %d completed the leader state transition for topic %s partition %d"
|
||||
.format(config.brokerId, replica.topic, replica.partition.partitionId))
|
||||
info("completed the leader state transition for topic %s partition %d".format(replica.topic, replica.partition.partitionId))
|
||||
ErrorMapping.NoError
|
||||
}catch {
|
||||
case e => error("Broker %d failed to complete the leader state transition for topic %s partition %d"
|
||||
.format(config.brokerId, replica.topic, replica.partition.partitionId), e)
|
||||
case e => error("failed to complete the leader state transition for topic %s partition %d".format(replica.topic, replica.partition.partitionId), e)
|
||||
ErrorMapping.UnknownCode
|
||||
/* TODO: add specific error code */
|
||||
}finally {
|
||||
leaderReplicaLock.unlock()
|
||||
}
|
||||
}
|
||||
|
||||
def makeFollower(replica: Replica, leaderBrokerId: Int, zkClient: ZkClient) {
|
||||
info("Broker %d starting the follower state transition to follow leader %d for topic %s partition %d"
|
||||
.format(config.brokerId, leaderBrokerId, replica.topic, replica.partition.partitionId))
|
||||
|
||||
def makeFollower(replica: Replica, leaderAndISR: LeaderAndISR): Short = {
|
||||
val leaderBrokerId: Int = leaderAndISR.leader
|
||||
info("starting the follower state transition to follow leader %d for topic %s partition %d"
|
||||
.format(leaderBrokerId, replica.topic, replica.partition.partitionId))
|
||||
try {
|
||||
// set the leader for this partition correctly on this broker
|
||||
replica.partition.leaderId(Some(leaderBrokerId))
|
||||
|
@ -177,13 +205,13 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
log.truncateTo(replica.highWatermark())
|
||||
case None =>
|
||||
}
|
||||
debug("for partition [%s, %d], the leaderBroker is [%d]".format(replica.topic, replica.partition.partitionId, leaderAndISR.leader))
|
||||
// get leader for this replica
|
||||
val leaderBroker = ZkUtils.getBrokerInfoFromIds(zkClient, List(leaderBrokerId)).head
|
||||
val currentLeaderBroker = replicaFetcherManager.fetcherSourceBroker(replica.topic, replica.partition.partitionId)
|
||||
// become follower only if it is not already following the same leader
|
||||
if( currentLeaderBroker == None || currentLeaderBroker.get != leaderBroker.id) {
|
||||
info("broker %d becoming follower to leader %d for topic %s partition %d"
|
||||
.format(config.brokerId, leaderBrokerId, replica.topic, replica.partition.partitionId))
|
||||
info("becoming follower to leader %d for topic %s partition %d".format(leaderBrokerId, replica.topic, replica.partition.partitionId))
|
||||
// stop fetcher thread to previous leader
|
||||
replicaFetcherManager.removeFetcher(replica.topic, replica.partition.partitionId)
|
||||
// start fetcher thread to current leader
|
||||
|
@ -192,11 +220,15 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
// remove this replica's partition from the ISR expiration queue
|
||||
leaderReplicaLock.lock()
|
||||
leaderReplicas -= replica.partition
|
||||
info("Broker %d completed the follower state transition to follow leader %d for topic %s partition %d"
|
||||
.format(config.brokerId, leaderBrokerId, replica.topic, replica.partition.partitionId))
|
||||
}catch {
|
||||
case e => error("Broker %d failed to complete the follower state transition to follow leader %d for topic %s partition %d"
|
||||
.format(config.brokerId, leaderBrokerId, replica.topic, replica.partition.partitionId), e)
|
||||
info("completed the follower state transition to follow leader %d for topic %s partition %d".format(leaderAndISR.leader, replica.topic, replica.partition.partitionId))
|
||||
ErrorMapping.NoError
|
||||
} catch {
|
||||
case e: BrokerNotExistException =>
|
||||
error("failed to complete the follower state transition to follow leader %d for topic %s partition %d because the leader broker does not exist in the cluster".format(leaderAndISR.leader, replica.topic, replica.partition.partitionId), e)
|
||||
ErrorMapping.BrokerNotExistInZookeeperCode
|
||||
case e =>
|
||||
error("failed to complete the follower state transition to follow leader %d for topic %s partition %d".format(leaderAndISR.leader, replica.topic, replica.partition.partitionId), e)
|
||||
ErrorMapping.UnknownCode
|
||||
}finally {
|
||||
leaderReplicaLock.unlock()
|
||||
}
|
||||
|
@ -204,21 +236,18 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
|
||||
private def maybeShrinkISR(): Unit = {
|
||||
try {
|
||||
info("Evaluating ISR list of partitions to see which replicas can be removed from the ISR"
|
||||
.format(config.replicaMaxLagTimeMs))
|
||||
info("evaluating ISR list of partitions to see which replicas can be removed from the ISR")
|
||||
leaderReplicaLock.lock()
|
||||
leaderReplicas.foreach { partition =>
|
||||
// shrink ISR if a follower is slow or stuck
|
||||
leaderReplicas.foreach(partition => {
|
||||
val outOfSyncReplicas = partition.getOutOfSyncReplicas(config.replicaMaxLagTimeMs, config.replicaMaxLagBytes)
|
||||
if(outOfSyncReplicas.size > 0) {
|
||||
val newInSyncReplicas = partition.inSyncReplicas -- outOfSyncReplicas
|
||||
assert(newInSyncReplicas.size > 0)
|
||||
info("Shrinking ISR for topic %s partition %d to %s".format(partition.topic, partition.partitionId,
|
||||
newInSyncReplicas.map(_.brokerId).mkString(",")))
|
||||
info("Shrinking ISR for topic %s partition %d to %s".format(partition.topic, partition.partitionId, newInSyncReplicas.map(_.brokerId).mkString(",")))
|
||||
// update ISR in zk and in memory
|
||||
partition.updateISR(newInSyncReplicas.map(_.brokerId), Some(zkClient))
|
||||
}
|
||||
}
|
||||
})
|
||||
}catch {
|
||||
case e1 => error("Error in ISR expiration thread. Shutting down due to ", e1)
|
||||
}finally {
|
||||
|
@ -233,8 +262,7 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
val leaderHW = partition.leaderHW()
|
||||
replica.logEndOffset() >= leaderHW
|
||||
}
|
||||
else throw new KafkaException("Replica %s is not in the assigned replicas list for ".format(replica.toString) +
|
||||
" topic %s partition %d on broker %d".format(replica.topic, replica.partition.partitionId, config.brokerId))
|
||||
else throw new KafkaException("Replica %s is not in the assigned replicas list for ".format(replica.toString) + " topic %s partition %d on broker %d".format(replica.topic, replica.partition.partitionId, config.brokerId))
|
||||
}
|
||||
|
||||
def recordFollowerPosition(topic: String, partition: Int, replicaId: Int, offset: Long, zkClient: ZkClient) = {
|
||||
|
@ -268,21 +296,21 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
* Flushes the highwatermark value for all partitions to the highwatermark file
|
||||
*/
|
||||
private def checkpointHighwaterMarks() {
|
||||
val highwaterMarksForAllPartitions = allReplicas.map { partition =>
|
||||
val topic = partition._1._1
|
||||
val partitionId = partition._1._2
|
||||
val localReplicaOpt = partition._2.getReplica(config.brokerId)
|
||||
val hw = localReplicaOpt match {
|
||||
case Some(localReplica) => localReplica.highWatermark()
|
||||
case None =>
|
||||
error("Error while checkpointing highwatermark for topic %s partition %d.".format(topic, partitionId) +
|
||||
" Replica metadata doesn't exist in replica manager on broker " + config.brokerId)
|
||||
0L
|
||||
}
|
||||
(topic, partitionId) -> hw
|
||||
}.toMap
|
||||
val highwaterMarksForAllPartitions = allPartitions.map
|
||||
{ partition =>
|
||||
val topic = partition._1._1
|
||||
val partitionId = partition._1._2
|
||||
val localReplicaOpt = partition._2.getReplica(config.brokerId)
|
||||
val hw = localReplicaOpt match {
|
||||
case Some(localReplica) => localReplica.highWatermark()
|
||||
case None =>
|
||||
error("Error while checkpointing highwatermark for topic %s partition %d.".format(topic, partitionId) + " Replica metadata doesn't exist")
|
||||
0L
|
||||
}
|
||||
(topic, partitionId) -> hw
|
||||
}.toMap
|
||||
highwaterMarkCheckpoint.write(highwaterMarksForAllPartitions)
|
||||
info("Checkpointed highwatermarks")
|
||||
info("Checkpointed high watermark data: %s".format(highwaterMarksForAllPartitions))
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -292,8 +320,9 @@ class ReplicaManager(val config: KafkaConfig, time: Time, zkClient: ZkClient, ka
|
|||
def readCheckpointedHighWatermark(topic: String, partition: Int): Long = highwaterMarkCheckpoint.read(topic, partition)
|
||||
|
||||
def shutdown() {
|
||||
info("shut down")
|
||||
replicaFetcherManager.shutdown()
|
||||
checkpointHighwaterMarks()
|
||||
info("Replica manager shutdown on broker " + config.brokerId)
|
||||
info("shuttedd down completely")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -39,7 +39,7 @@ class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, de
|
|||
* request to be satisfied. For example it could be that we are waiting for user-specified number of acks on a given (topic, partition)
|
||||
* to be able to respond to a request or it could be that we are waiting for a given number of bytes to accumulate on a given request
|
||||
* to be able to respond to that request (in the simple case we might wait for at least one byte to avoid busy waiting).
|
||||
*
|
||||
*
|
||||
* For us the key is generally a (topic, partition) pair.
|
||||
* By calling
|
||||
* watch(delayedRequest)
|
||||
|
@ -47,27 +47,27 @@ class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, de
|
|||
* val satisfied = update(key, request)
|
||||
* when a request relevant to the given key occurs. This triggers bookeeping logic and returns back any requests satisfied by this
|
||||
* new request.
|
||||
*
|
||||
*
|
||||
* An implementation provides extends two helper functions
|
||||
* def checkSatisfied(request: R, delayed: T): Boolean
|
||||
* this function returns true if the given request (in combination with whatever previous requests have happened) satisfies the delayed
|
||||
* request delayed. This method will likely also need to do whatever bookkeeping is necessary.
|
||||
*
|
||||
*
|
||||
* The second function is
|
||||
* def expire(delayed: T)
|
||||
* this function handles delayed requests that have hit their time limit without being satisfied.
|
||||
*
|
||||
*
|
||||
*/
|
||||
abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
||||
|
||||
abstract class RequestPurgatory[T <: DelayedRequest, R](logPrefix: String) extends Logging{
|
||||
this.logIdent = logPrefix
|
||||
/* a list of requests watching each key */
|
||||
private val watchersForKey = new ConcurrentHashMap[Any, Watchers]
|
||||
|
||||
|
||||
/* background thread expiring requests that have been waiting too long */
|
||||
private val expiredRequestReaper = new ExpiredRequestReaper
|
||||
private val expiredRequestReaper = new ExpiredRequestReaper(logPrefix)
|
||||
private val expirationThread = Utils.daemonThread("request-expiration-task", expiredRequestReaper)
|
||||
expirationThread.start()
|
||||
|
||||
|
||||
/**
|
||||
* Add a new delayed request watching the contained keys
|
||||
*/
|
||||
|
@ -78,7 +78,7 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
}
|
||||
expiredRequestReaper.enqueue(delayedRequest)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Update any watchers and return a list of newly satisfied requests.
|
||||
*/
|
||||
|
@ -89,7 +89,7 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
else
|
||||
w.collectSatisfiedRequests(request)
|
||||
}
|
||||
|
||||
|
||||
private def watchersFor(key: Any): Watchers = {
|
||||
var lst = watchersForKey.get(key)
|
||||
if(lst == null) {
|
||||
|
@ -98,46 +98,46 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
}
|
||||
lst
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Check if this request satisfied this delayed request
|
||||
*/
|
||||
protected def checkSatisfied(request: R, delayed: T): Boolean
|
||||
|
||||
|
||||
/**
|
||||
* Handle an expired delayed request
|
||||
*/
|
||||
protected def expire(delayed: T)
|
||||
|
||||
|
||||
/**
|
||||
* Shutdown the expirey thread
|
||||
*/
|
||||
def shutdown() {
|
||||
expiredRequestReaper.shutdown()
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A linked list of DelayedRequests watching some key with some associated bookeeping logic
|
||||
*/
|
||||
private class Watchers {
|
||||
|
||||
|
||||
/* a few magic parameters to help do cleanup to avoid accumulating old watchers */
|
||||
private val CleanupThresholdSize = 100
|
||||
private val CleanupThresholdPrct = 0.5
|
||||
|
||||
|
||||
private val requests = new LinkedList[T]
|
||||
|
||||
|
||||
/* you can only change this if you have added something or marked something satisfied */
|
||||
var liveCount = 0.0
|
||||
|
||||
|
||||
def add(t: T) {
|
||||
synchronized {
|
||||
requests.add(t)
|
||||
liveCount += 1
|
||||
maybePurge()
|
||||
}
|
||||
requests.add(t)
|
||||
liveCount += 1
|
||||
maybePurge()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private def maybePurge() {
|
||||
if(requests.size > CleanupThresholdSize && liveCount / requests.size < CleanupThresholdPrct) {
|
||||
val iter = requests.iterator()
|
||||
|
@ -148,55 +148,56 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
def decLiveCount() {
|
||||
synchronized {
|
||||
liveCount -= 1
|
||||
}
|
||||
liveCount -= 1
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
def collectSatisfiedRequests(request: R): Seq[T] = {
|
||||
val response = new mutable.ArrayBuffer[T]
|
||||
synchronized {
|
||||
val iter = requests.iterator()
|
||||
while(iter.hasNext) {
|
||||
val curr = iter.next
|
||||
if(curr.satisfied.get) {
|
||||
// another thread has satisfied this request, remove it
|
||||
iter.remove()
|
||||
} else {
|
||||
if(checkSatisfied(request, curr)) {
|
||||
iter.remove()
|
||||
val updated = curr.satisfied.compareAndSet(false, true)
|
||||
if(updated == true) {
|
||||
response += curr
|
||||
liveCount -= 1
|
||||
expiredRequestReaper.satisfyRequest()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
val iter = requests.iterator()
|
||||
while(iter.hasNext) {
|
||||
val curr = iter.next
|
||||
if(curr.satisfied.get) {
|
||||
// another thread has satisfied this request, remove it
|
||||
iter.remove()
|
||||
} else {
|
||||
if(checkSatisfied(request, curr)) {
|
||||
iter.remove()
|
||||
val updated = curr.satisfied.compareAndSet(false, true)
|
||||
if(updated == true) {
|
||||
response += curr
|
||||
liveCount -= 1
|
||||
expiredRequestReaper.satisfyRequest()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
response
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Runnable to expire requests that have sat unfullfilled past their deadline
|
||||
*/
|
||||
private class ExpiredRequestReaper extends Runnable with Logging {
|
||||
|
||||
private class ExpiredRequestReaper(logPrefix: String) extends Runnable with Logging {
|
||||
this.logIdent = "ExpiredRequestReaper for " + logPrefix
|
||||
|
||||
/* a few magic parameters to help do cleanup to avoid accumulating old watchers */
|
||||
private val CleanupThresholdSize = 100
|
||||
private val CleanupThresholdPrct = 0.5
|
||||
|
||||
|
||||
private val delayed = new DelayQueue[T]
|
||||
private val running = new AtomicBoolean(true)
|
||||
private val shutdownLatch = new CountDownLatch(1)
|
||||
private val needsPurge = new AtomicBoolean(false)
|
||||
/* The count of elements in the delay queue that are unsatisfied */
|
||||
private val unsatisfied = new AtomicInteger(0)
|
||||
|
||||
|
||||
/** Main loop for the expiry thread */
|
||||
def run() {
|
||||
while(running.get) {
|
||||
|
@ -204,18 +205,18 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
val curr = pollExpired()
|
||||
expire(curr)
|
||||
} catch {
|
||||
case ie: InterruptedException =>
|
||||
case ie: InterruptedException =>
|
||||
if(needsPurge.getAndSet(false)) {
|
||||
val purged = purgeSatisfied()
|
||||
debug("Forced purge of " + purged + " requests from delay queue.")
|
||||
}
|
||||
case e: Exception =>
|
||||
case e: Exception =>
|
||||
error("Error in long poll expiry thread: ", e)
|
||||
}
|
||||
}
|
||||
shutdownLatch.countDown()
|
||||
}
|
||||
|
||||
|
||||
/** Add a request to be expired */
|
||||
def enqueue(t: T) {
|
||||
delayed.add(t)
|
||||
|
@ -223,23 +224,24 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
if(unsatisfied.get > CleanupThresholdSize && unsatisfied.get / delayed.size.toDouble < CleanupThresholdPrct)
|
||||
forcePurge()
|
||||
}
|
||||
|
||||
|
||||
private def forcePurge() {
|
||||
needsPurge.set(true)
|
||||
expirationThread.interrupt()
|
||||
}
|
||||
|
||||
|
||||
/** Shutdown the expiry thread*/
|
||||
def shutdown() {
|
||||
debug("Shutting down request expiry thread")
|
||||
debug("shutting down")
|
||||
running.set(false)
|
||||
expirationThread.interrupt()
|
||||
shutdownLatch.await()
|
||||
debug("shut down completely")
|
||||
}
|
||||
|
||||
|
||||
/** Record the fact that we satisfied a request in the stats for the expiry queue */
|
||||
def satisfyRequest(): Unit = unsatisfied.getAndDecrement()
|
||||
|
||||
|
||||
/**
|
||||
* Get the next expired event
|
||||
*/
|
||||
|
@ -256,7 +258,7 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
}
|
||||
throw new RuntimeException("This should not happen")
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Delete all expired events from the delay queue
|
||||
*/
|
||||
|
@ -273,5 +275,5 @@ abstract class RequestPurgatory[T <: DelayedRequest, R] {
|
|||
purged
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -1,92 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.server
|
||||
|
||||
import util.parsing.json.JSON
|
||||
import kafka.common.KafkaException
|
||||
import kafka.utils.{Utils, Logging}
|
||||
import collection.mutable.HashMap
|
||||
|
||||
object StateChangeCommand extends Logging {
|
||||
val State = "state"
|
||||
val Topic = "topic"
|
||||
val Partition = "partition"
|
||||
val Epoch = "epoch"
|
||||
val StartReplica = "start-replica"
|
||||
val CloseReplica = "close-replica"
|
||||
|
||||
def getStateChangeRequest(requestJson: String): StateChangeCommand = {
|
||||
var topMap : Map[String, String] = null
|
||||
try {
|
||||
JSON.parseFull(requestJson) match {
|
||||
case Some(m) =>
|
||||
topMap = m.asInstanceOf[Map[String, String]]
|
||||
val topic = topMap.get(StateChangeCommand.Topic).getOrElse(null)
|
||||
val partition = topMap.get(StateChangeCommand.Partition).getOrElse("-1").toInt
|
||||
val epoch = topMap.get(StateChangeCommand.Epoch).getOrElse("-1").toInt
|
||||
val requestOpt = topMap.get(StateChangeCommand.State)
|
||||
requestOpt match {
|
||||
case Some(request) =>
|
||||
request match {
|
||||
case StartReplica => new StartReplica(topic, partition, epoch)
|
||||
case CloseReplica => new CloseReplica(topic, partition, epoch)
|
||||
case _ => throw new KafkaException("Unknown state change request " + request)
|
||||
}
|
||||
case None =>
|
||||
throw new KafkaException("Illegal state change request JSON " + requestJson)
|
||||
}
|
||||
case None => throw new RuntimeException("Error parsing state change request : " + requestJson)
|
||||
}
|
||||
} catch {
|
||||
case e =>
|
||||
error("Error parsing state change request JSON " + requestJson, e)
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
sealed trait StateChangeCommand extends Logging {
|
||||
def state: String
|
||||
|
||||
def topic: String
|
||||
|
||||
def partition: Int
|
||||
|
||||
def epoch: Int
|
||||
|
||||
def toJson(): String = {
|
||||
val jsonMap = new HashMap[String, String]
|
||||
jsonMap.put(StateChangeCommand.State, state)
|
||||
jsonMap.put(StateChangeCommand.Topic, topic)
|
||||
jsonMap.put(StateChangeCommand.Partition, partition.toString)
|
||||
jsonMap.put(StateChangeCommand.Epoch, epoch.toString)
|
||||
Utils.stringMapToJsonString(jsonMap)
|
||||
}
|
||||
}
|
||||
|
||||
/* The elected leader sends the start replica state change request to all the new replicas that have been assigned
|
||||
* a partition. Note that the followers must act on this request only if the request epoch == latest partition epoch or -1 */
|
||||
case class StartReplica(val topic: String, partition: Int, epoch: Int) extends StateChangeCommand {
|
||||
val state: String = StateChangeCommand.StartReplica
|
||||
}
|
||||
|
||||
/* The elected leader sends the close replica state change request to all the replicas that have been un-assigned a partition
|
||||
* OR if a topic has been deleted. Note that the followers must act on this request even if the epoch has changed */
|
||||
case class CloseReplica(topic: String, partition: Int, epoch: Int) extends StateChangeCommand {
|
||||
val state: String = StateChangeCommand.CloseReplica
|
||||
}
|
|
@ -1,79 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.server
|
||||
|
||||
import kafka.utils.{ZkQueue, Logging}
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.CountDownLatch
|
||||
|
||||
class StateChangeCommandHandler(name: String, config: KafkaConfig, stateChangeQ: ZkQueue,
|
||||
ensureStateChangeCommandValidityOnThisBroker: (StateChangeCommand) => Boolean,
|
||||
ensureEpochValidity: (StateChangeCommand) => Boolean) extends Thread(name) with Logging {
|
||||
val isRunning: AtomicBoolean = new AtomicBoolean(true)
|
||||
private val shutdownLatch = new CountDownLatch(1)
|
||||
|
||||
override def run() {
|
||||
try {
|
||||
while(isRunning.get()) {
|
||||
// get outstanding state change requests for this broker
|
||||
val command = stateChangeQ.take()
|
||||
val stateChangeCommand = StateChangeCommand.getStateChangeRequest(command._2)
|
||||
ensureStateChangeCommandValidityOnThisBroker(stateChangeCommand)
|
||||
|
||||
stateChangeCommand match {
|
||||
case StartReplica(topic, partition, epoch) =>
|
||||
if(ensureEpochValidity(stateChangeCommand))
|
||||
handleStartReplica(topic, partition)
|
||||
case CloseReplica(topic, partition, epoch) =>
|
||||
/**
|
||||
* close replica requests are sent as part of delete topic or partition reassignment process
|
||||
* To ensure that a topic will be deleted even if the broker is offline, this state change should not
|
||||
* be protected with the epoch validity check
|
||||
*/
|
||||
handleCloseReplica(topic, partition)
|
||||
}
|
||||
stateChangeQ.remove(command)
|
||||
}
|
||||
}catch {
|
||||
case e: InterruptedException => info("State change command handler interrupted. Shutting down")
|
||||
case e1 => error("Error in state change command handler. Shutting down due to ", e1)
|
||||
}
|
||||
shutdownComplete()
|
||||
}
|
||||
|
||||
private def shutdownComplete() = shutdownLatch.countDown
|
||||
|
||||
def shutdown() {
|
||||
isRunning.set(false)
|
||||
interrupt()
|
||||
shutdownLatch.await()
|
||||
info("State change command handler shutdown completed")
|
||||
}
|
||||
|
||||
def handleStartReplica(topic: String, partition: Int) {
|
||||
info("Received start replica state change command for topic %s partition %d on broker %d"
|
||||
.format(topic, partition, config.brokerId))
|
||||
// TODO: implement this as part of create topic support or partition reassignment support. Until then, it is unused
|
||||
}
|
||||
|
||||
def handleCloseReplica(topic: String, partition: Int) {
|
||||
info("Received close replica state change command for topic %s partition %d on broker %d"
|
||||
.format(topic, partition, config.brokerId))
|
||||
// TODO: implement this as part of delete topic support. Until then, it is unused
|
||||
}
|
||||
}
|
|
@ -33,7 +33,7 @@ object ConsumerOffsetChecker extends Logging {
|
|||
// e.g., 127.0.0.1-1315436360737:127.0.0.1:9092
|
||||
|
||||
private def getConsumer(zkClient: ZkClient, bid: String): Option[SimpleConsumer] = {
|
||||
val brokerInfo = ZkUtils.readDataMaybeNull(zkClient, "/brokers/ids/%s".format(bid))
|
||||
val brokerInfo = ZkUtils.readDataMaybeNull(zkClient, "/brokers/ids/%s".format(bid))._1
|
||||
val consumer = brokerInfo match {
|
||||
case BrokerIpPattern(ip, port) =>
|
||||
Some(new SimpleConsumer(ip, port.toInt, 10000, 100000))
|
||||
|
@ -47,9 +47,9 @@ object ConsumerOffsetChecker extends Logging {
|
|||
private def processPartition(zkClient: ZkClient,
|
||||
group: String, topic: String, bidPid: String) {
|
||||
val offset = ZkUtils.readData(zkClient, "/consumers/%s/offsets/%s/%s".
|
||||
format(group, topic, bidPid)).toLong
|
||||
format(group, topic, bidPid))._1.toLong
|
||||
val owner = ZkUtils.readDataMaybeNull(zkClient, "/consumers/%s/owners/%s/%s".
|
||||
format(group, topic, bidPid))
|
||||
format(group, topic, bidPid))._1
|
||||
println("%s,%s,%s (Group,Topic,BrokerId-PartitionId)".format(group, topic, bidPid))
|
||||
println("%20s%s".format("Owner = ", owner))
|
||||
println("%20s%d".format("Consumer offset = ", offset))
|
||||
|
|
|
@ -100,7 +100,7 @@ object ExportZkOffsets extends Logging {
|
|||
for (bidPid <- bidPidList) {
|
||||
val zkGrpTpDir = new ZKGroupTopicDirs(consumerGrp,topic)
|
||||
val offsetPath = zkGrpTpDir.consumerOffsetDir + "/" + bidPid
|
||||
val offsetVal = ZkUtils.readDataMaybeNull(zkClient, offsetPath)
|
||||
val offsetVal = ZkUtils.readDataMaybeNull(zkClient, offsetPath)._1
|
||||
fileWriter.write(offsetPath + ":" + offsetVal + "\n")
|
||||
debug(offsetPath + " => " + offsetVal)
|
||||
}
|
||||
|
|
|
@ -104,7 +104,7 @@ object VerifyConsumerRebalance extends Logging {
|
|||
}
|
||||
// try reading the partition owner path for see if a valid consumer id exists there
|
||||
val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition
|
||||
val partitionOwner = ZkUtils.readDataMaybeNull(zkClient, partitionOwnerPath)
|
||||
val partitionOwner = ZkUtils.readDataMaybeNull(zkClient, partitionOwnerPath)._1
|
||||
if(partitionOwner == null) {
|
||||
error("No owner for topic %s partition %s".format(topic, partition))
|
||||
rebalanceSucceeded = false
|
||||
|
|
|
@ -45,7 +45,7 @@ object UpdateOffsetsInZK {
|
|||
private def getAndSetOffsets(zkClient: ZkClient, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = {
|
||||
val cluster = ZkUtils.getCluster(zkClient)
|
||||
val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, List(topic).iterator)
|
||||
var partitions: Seq[String] = Nil
|
||||
var partitions: Seq[Int] = Nil
|
||||
|
||||
partitionsPerTopicMap.get(topic) match {
|
||||
case Some(l) => partitions = l.sortWith((s,t) => s < t)
|
||||
|
@ -54,7 +54,7 @@ object UpdateOffsetsInZK {
|
|||
|
||||
var numParts = 0
|
||||
for (partition <- partitions) {
|
||||
val brokerHostingPartition = ZkUtils.getLeaderForPartition(zkClient, topic, partition.toInt)
|
||||
val brokerHostingPartition = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
|
||||
|
||||
val broker = brokerHostingPartition match {
|
||||
case Some(b) => b
|
||||
|
@ -68,7 +68,7 @@ object UpdateOffsetsInZK {
|
|||
|
||||
val brokerInfo = brokerInfos.head
|
||||
val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024)
|
||||
val offsets = consumer.getOffsetsBefore(topic, partition.toInt, offsetOption, 1)
|
||||
val offsets = consumer.getOffsetsBefore(topic, partition, offsetOption, 1)
|
||||
val topicDirs = new ZKGroupTopicDirs(config.groupId, topic)
|
||||
|
||||
println("updating partition " + partition + " with new offset: " + offsets(0))
|
||||
|
|
|
@ -1,127 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.utils
|
||||
|
||||
import kafka.utils.ZkUtils._
|
||||
import kafka.common.QueueFullException
|
||||
import org.I0Itec.zkclient.{IZkChildListener, ZkClient}
|
||||
import java.util.concurrent.PriorityBlockingQueue
|
||||
import java.util.Comparator
|
||||
|
||||
class ZkQueue(zkClient: ZkClient, path: String, size: Int) {
|
||||
// create the queue in ZK, if one does not exist
|
||||
makeSurePersistentPathExists(zkClient, path)
|
||||
val queueItems = new PriorityBlockingQueue[String](size, new ZkQueueComparator)
|
||||
var latestQueueItemPriority: Int = -1
|
||||
zkClient.subscribeChildChanges(path, new ZkQueueListener)
|
||||
|
||||
// TODO: This API will be used by the leader to enqueue state change requests to the followers
|
||||
/**
|
||||
* Inserts the specified element into this priority queue. This method will never block. If the queue is full,
|
||||
* it will throw QueueFullException
|
||||
* @param item Item to add to the zookeeper queue
|
||||
* @returns The zookeeper location of item in the queue
|
||||
*/
|
||||
def put(item: String): String = {
|
||||
// if queue is full, throw QueueFullException
|
||||
if(isFull)
|
||||
throw new QueueFullException("Queue is full. Item %s will be rejected".format(item))
|
||||
val queueLocation = createSequentialPersistentPath(zkClient, path + "/", item)
|
||||
debug("Added item %s to queue at location %s".format(item, queueLocation))
|
||||
queueLocation
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads all the items and their queue locations in this queue
|
||||
* @returns A list of (queue_location, item) pairs
|
||||
*/
|
||||
def readAll(): Seq[(String, String)] = {
|
||||
val allItems = getChildren(zkClient, path).sorted
|
||||
allItems.size match {
|
||||
case 0 => Seq.empty[(String, String)]
|
||||
case _ => allItems.map { item =>
|
||||
// read the data and delete the node
|
||||
val queueLocation = path + "/" + item
|
||||
val data = ZkUtils.readData(zkClient, queueLocation)
|
||||
(item, data)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if this zookeeper queue contains no elements.
|
||||
*/
|
||||
def isEmpty: Boolean = (readAll().size == 0)
|
||||
|
||||
// TODO: Implement the queue shrink operation if the queue is full, as part of create/delete topic
|
||||
/**
|
||||
* Returns true if this zookeeper queue contains number of items equal to the size of the queue
|
||||
*/
|
||||
def isFull: Boolean = (readAll().size == size)
|
||||
|
||||
/**
|
||||
* Retrieves but does not remove the head of this queue, waiting if necessary until an element becomes available.
|
||||
* @returns The location of the head and the head element in the zookeeper queue
|
||||
*/
|
||||
def take(): (String, String) = {
|
||||
// take the element key
|
||||
val item = queueItems.take()
|
||||
val queueLocation = path + "/" + item
|
||||
val data = ZkUtils.readData(zkClient, queueLocation)
|
||||
(item, data)
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes a single instance of the specified element from this queue, if it is present. More formally, removes an
|
||||
* element e such that o.equals(e), if this queue contains one or more such elements. Returns true if this queue
|
||||
* contained the specified element (or equivalently, if this queue changed as a result of the call).
|
||||
* @param queueItem A tuple where the first element is the location of the item as returned by the take() API and the
|
||||
* second element is the queue item to be removed
|
||||
*/
|
||||
def remove(queueItem: (String, String)): Boolean = {
|
||||
val queueLocation = path + "/" + queueItem._1
|
||||
// we do not want to remove items from the queue if they were not read
|
||||
assert(!queueItems.contains(queueItem._1), "Attempt to remove unconsumed item %s from the queue".format(queueItem))
|
||||
ZkUtils.deletePath(zkClient, queueLocation)
|
||||
}
|
||||
|
||||
class ZkQueueListener extends IZkChildListener with Logging {
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleChildChange(parentPath : String, curChilds : java.util.List[String]) {
|
||||
debug("ZkQueue listener fired for queue %s with children %s and latest queue item priority %d"
|
||||
.format(path, curChilds.toString, latestQueueItemPriority))
|
||||
import scala.collection.JavaConversions._
|
||||
val outstandingRequests = asBuffer(curChilds).sortWith((req1, req2) => req1.toInt < req2.toInt)
|
||||
outstandingRequests.foreach { req =>
|
||||
val queueItemPriority = req.toInt
|
||||
if(queueItemPriority > latestQueueItemPriority) {
|
||||
latestQueueItemPriority = queueItemPriority
|
||||
queueItems.add(req)
|
||||
debug("Added item %s to queue %s".format(req, path))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ZkQueueComparator extends Comparator[String] {
|
||||
def compare(element1: String, element2: String): Int = {
|
||||
element1.toInt - element2.toInt
|
||||
}
|
||||
}
|
||||
}
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -19,22 +19,29 @@ package kafka.utils
|
|||
|
||||
import java.util.Properties
|
||||
import kafka.cluster.{Broker, Cluster}
|
||||
import kafka.common.NoEpochForPartitionException
|
||||
import kafka.consumer.TopicCount
|
||||
import org.I0Itec.zkclient.{IZkDataListener, ZkClient}
|
||||
import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError}
|
||||
import org.I0Itec.zkclient.serialize.ZkSerializer
|
||||
import scala.collection._
|
||||
import util.parsing.json.JSON
|
||||
import kafka.api.LeaderAndISR
|
||||
import kafka.common.NoEpochForPartitionException
|
||||
import org.apache.zookeeper.data.Stat
|
||||
import java.util.concurrent.locks.{ReentrantLock, Condition}
|
||||
import scala.throws
|
||||
|
||||
|
||||
object ZkUtils extends Logging {
|
||||
val ConsumersPath = "/consumers"
|
||||
val BrokerIdsPath = "/brokers/ids"
|
||||
val BrokerTopicsPath = "/brokers/topics"
|
||||
val BrokerStatePath = "/brokers/state"
|
||||
val ControllerPath = "/controller"
|
||||
|
||||
def getBrokerPath(brokerId: Int): String = {
|
||||
BrokerIdsPath + "/" + brokerId
|
||||
}
|
||||
|
||||
def getTopicPath(topic: String): String ={
|
||||
BrokerTopicsPath + "/" + topic
|
||||
}
|
||||
|
@ -44,53 +51,63 @@ object ZkUtils extends Logging {
|
|||
}
|
||||
|
||||
def getController(zkClient: ZkClient): Int= {
|
||||
val controller = readDataMaybeNull(zkClient, ControllerPath)
|
||||
val controller = readDataMaybeNull(zkClient, ControllerPath)._1
|
||||
controller.toInt
|
||||
}
|
||||
|
||||
def getTopicPartitionPath(topic: String, partitionId: String): String ={
|
||||
def getTopicPartitionPath(topic: String, partitionId: Int): String ={
|
||||
getTopicPartitionsPath(topic) + "/" + partitionId
|
||||
}
|
||||
|
||||
def getTopicPartitionLeaderAndISR(topic: String, partitionId: String): String ={
|
||||
def getTopicPartitionLeaderAndISRPath(topic: String, partitionId: Int): String ={
|
||||
getTopicPartitionPath(topic, partitionId) + "/" + "leaderAndISR"
|
||||
}
|
||||
|
||||
def getTopicVersion(zkClient: ZkClient, topic: String): String ={
|
||||
readDataMaybeNull(zkClient, getTopicPath(topic))
|
||||
}
|
||||
|
||||
def getTopicPartitionReplicasPath(topic: String, partitionId: String): String ={
|
||||
getTopicPartitionPath(topic, partitionId) + "/" + "replicas"
|
||||
}
|
||||
|
||||
def getTopicPartitionInSyncPath(topic: String, partitionId: String): String ={
|
||||
getTopicPartitionPath(topic, partitionId) + "/" + "isr"
|
||||
}
|
||||
|
||||
def getTopicPartitionLeaderPath(topic: String, partitionId: String): String ={
|
||||
getTopicPartitionPath(topic, partitionId) + "/" + "leader"
|
||||
}
|
||||
|
||||
def getBrokerStateChangePath(brokerId: Int): String = {
|
||||
BrokerStatePath + "/" + brokerId
|
||||
}
|
||||
|
||||
def getSortedBrokerList(zkClient: ZkClient): Seq[String] ={
|
||||
ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted
|
||||
ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted
|
||||
}
|
||||
|
||||
def getAllLiveBrokerIds(zkClient: ZkClient): Set[Int] = {
|
||||
ZkUtils.getChildren(zkClient, BrokerIdsPath).map(_.toInt).toSet
|
||||
}
|
||||
|
||||
def getAllBrokersInCluster(zkClient: ZkClient): Seq[Broker] = {
|
||||
val brokerIds = ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted
|
||||
getBrokerInfoFromIds(zkClient, brokerIds.map(b => b.toInt))
|
||||
getBrokerInfoFromIds(zkClient, brokerIds.map(_.toInt))
|
||||
}
|
||||
|
||||
def getLeaderForPartition(zkClient: ZkClient, topic: String, partition: Int): Option[Int] = {
|
||||
val leaderAndEpoch = readDataMaybeNull(zkClient, getTopicPartitionLeaderPath(topic, partition.toString))
|
||||
if(leaderAndEpoch == null) None
|
||||
|
||||
def getLeaderAndISRForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndISR] = {
|
||||
val leaderAndISRPath = getTopicPartitionLeaderAndISRPath(topic, partition)
|
||||
val ret = readDataMaybeNull(zkClient, leaderAndISRPath)
|
||||
val leaderAndISRStr: String = ret._1
|
||||
val stat = ret._2
|
||||
if(leaderAndISRStr == null) None
|
||||
else {
|
||||
val leaderAndEpochInfo = leaderAndEpoch.split(";")
|
||||
Some(leaderAndEpochInfo.head.toInt)
|
||||
JSON.parseFull(leaderAndISRStr) match {
|
||||
case Some(m) =>
|
||||
val leader = m.asInstanceOf[Map[String, String]].get("leader").get.toInt
|
||||
val epoch = m.asInstanceOf[Map[String, String]].get("leaderEpoch").get.toInt
|
||||
val ISRString = m.asInstanceOf[Map[String, String]].get("ISR").get
|
||||
val ISR = Utils.getCSVList(ISRString).map(r => r.toInt)
|
||||
val zkPathVersion = stat.getVersion
|
||||
debug("Leader %d, Epoch %d, isr %s, zk path version %d for topic %s and partition %d".format(leader, epoch, ISR.toString(), zkPathVersion, topic, partition))
|
||||
Some(LeaderAndISR(leader, epoch, ISR.toList, zkPathVersion))
|
||||
case None => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
def getLeaderForPartition(zkClient: ZkClient, topic: String, partition: Int): Option[Int] = {
|
||||
val leaderAndISR = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndISRPath(topic, partition))._1
|
||||
if(leaderAndISR == null) None
|
||||
else {
|
||||
JSON.parseFull(leaderAndISR) match {
|
||||
case Some(m) =>
|
||||
Some(m.asInstanceOf[Map[String, String]].get("leader").get.toInt)
|
||||
case None => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -99,78 +116,62 @@ object ZkUtils extends Logging {
|
|||
* leader fails after updating epoch in the leader path and before updating epoch in the ISR path, effectively some
|
||||
* other broker will retry becoming leader with the same new epoch value.
|
||||
*/
|
||||
def getEpochForPartition(client: ZkClient, topic: String, partition: Int): Int = {
|
||||
val lastKnownEpoch = try {
|
||||
val isrAndEpoch = readData(client, getTopicPartitionInSyncPath(topic, partition.toString))
|
||||
if(isrAndEpoch != null) {
|
||||
val isrAndEpochInfo = isrAndEpoch.split(";")
|
||||
if(isrAndEpochInfo.last.isEmpty)
|
||||
throw new NoEpochForPartitionException("No epoch in ISR path for topic %s partition %d is empty".format(topic, partition))
|
||||
else
|
||||
isrAndEpochInfo.last.toInt
|
||||
}else {
|
||||
throw new NoEpochForPartitionException("ISR path for topic %s partition %d is empty".format(topic, partition))
|
||||
def getEpochForPartition(zkClient: ZkClient, topic: String, partition: Int): Int = {
|
||||
val leaderAndISR = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndISRPath(topic, partition))._1
|
||||
if(leaderAndISR != null) {
|
||||
val epoch = JSON.parseFull(leaderAndISR) match {
|
||||
case None => throw new NoEpochForPartitionException("No epoch, leaderAndISR data for topic %s partition %d is invalid".format(topic, partition))
|
||||
case Some(m) =>
|
||||
m.asInstanceOf[Map[String, String]].get("leaderEpoch").get.toInt
|
||||
}
|
||||
} catch {
|
||||
case e: ZkNoNodeException =>
|
||||
throw new NoEpochForPartitionException("No epoch since leader never existed for topic %s partition %d".format(topic, partition))
|
||||
case e1 => throw e1
|
||||
}
|
||||
lastKnownEpoch
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the assigned replicas (AR) for a specific topic and partition
|
||||
*/
|
||||
def getReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[String] = {
|
||||
val topicAndPartitionAssignment = getPartitionAssignmentForTopics(zkClient, List(topic).iterator)
|
||||
topicAndPartitionAssignment.get(topic) match {
|
||||
case Some(partitionAssignment) => partitionAssignment.get(partition.toString) match {
|
||||
case Some(replicaList) => replicaList
|
||||
case None => Seq.empty[String]
|
||||
}
|
||||
case None => Seq.empty[String]
|
||||
epoch
|
||||
}
|
||||
else
|
||||
throw new NoEpochForPartitionException("No epoch, ISR path for topic %s partition %d is empty".format(topic, partition))
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the in-sync replicas (ISR) for a specific topic and partition
|
||||
*/
|
||||
def getInSyncReplicasForPartition(client: ZkClient, topic: String, partition: Int): Seq[Int] = {
|
||||
val replicaListAndEpochString = readDataMaybeNull(client, getTopicPartitionInSyncPath(topic, partition.toString))
|
||||
if(replicaListAndEpochString == null)
|
||||
Seq.empty[Int]
|
||||
def getInSyncReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[Int] = {
|
||||
val leaderAndISR = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndISRPath(topic, partition))._1
|
||||
if(leaderAndISR == null) Seq.empty[Int]
|
||||
else {
|
||||
val replicasAndEpochInfo = replicaListAndEpochString.split(";")
|
||||
Utils.getCSVList(replicasAndEpochInfo.head).map(r => r.toInt)
|
||||
JSON.parseFull(leaderAndISR) match {
|
||||
case Some(m) =>
|
||||
val ISRString = m.asInstanceOf[Map[String, String]].get("ISR").get
|
||||
Utils.getCSVList(ISRString).map(r => r.toInt)
|
||||
case None => Seq.empty[Int]
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the assigned replicas (AR) for a specific topic and partition
|
||||
*/
|
||||
def getReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[Int] = {
|
||||
val jsonPartitionMap = readDataMaybeNull(zkClient, getTopicPath(topic))._1
|
||||
val assignedReplicas = if (jsonPartitionMap == null) {
|
||||
Seq.empty[Int]
|
||||
} else {
|
||||
JSON.parseFull(jsonPartitionMap) match {
|
||||
case Some(m) => m.asInstanceOf[Map[String, List[String]]].get(partition.toString) match {
|
||||
case None => Seq.empty[Int]
|
||||
case Some(seq) => seq.map(_.toInt)
|
||||
}
|
||||
case None => Seq.empty[Int]
|
||||
}
|
||||
}
|
||||
assignedReplicas
|
||||
}
|
||||
|
||||
def isPartitionOnBroker(zkClient: ZkClient, topic: String, partition: Int, brokerId: Int): Boolean = {
|
||||
val replicas = getReplicasForPartition(zkClient, topic, partition)
|
||||
debug("The list of replicas for topic %s, partition %d is %s".format(topic, partition, replicas))
|
||||
replicas.contains(brokerId.toString)
|
||||
}
|
||||
|
||||
def tryToBecomeLeaderForPartition(client: ZkClient, topic: String, partition: Int, brokerId: Int): Option[(Int, Seq[Int])] = {
|
||||
try {
|
||||
// NOTE: first increment epoch, then become leader
|
||||
val newEpoch = incrementEpochForPartition(client, topic, partition, brokerId)
|
||||
createEphemeralPathExpectConflict(client, getTopicPartitionLeaderPath(topic, partition.toString),
|
||||
"%d;%d".format(brokerId, newEpoch))
|
||||
val currentISR = getInSyncReplicasForPartition(client, topic, partition)
|
||||
val updatedISR = if(currentISR.size == 0) List(brokerId) else currentISR
|
||||
updatePersistentPath(client, getTopicPartitionInSyncPath(topic, partition.toString),
|
||||
"%s;%d".format(updatedISR.mkString(","), newEpoch))
|
||||
info("Elected broker %d with epoch %d to be leader for topic %s partition %d".format(brokerId, newEpoch, topic, partition))
|
||||
Some(newEpoch, updatedISR)
|
||||
} catch {
|
||||
case e: ZkNodeExistsException => error("Leader exists for topic %s partition %d".format(topic, partition)); None
|
||||
case oe => error("Error while electing leader for topic %s partition %d".format(topic, partition), oe); None
|
||||
}
|
||||
}
|
||||
|
||||
def incrementEpochForPartition(client: ZkClient, topic: String, partition: Int, leader: Int) = {
|
||||
def incrementEpochForPartition(client: ZkClient, topic: String, partition: Int, leader: Int): Int = {
|
||||
// read previous epoch, increment it and write it to the leader path and the ISR path.
|
||||
val epoch = try {
|
||||
Some(getEpochForPartition(client, topic, partition))
|
||||
|
@ -198,15 +199,12 @@ object ZkUtils extends Logging {
|
|||
createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZKString)
|
||||
} catch {
|
||||
case e: ZkNodeExistsException =>
|
||||
throw new RuntimeException("A broker is already registered on the path " + brokerIdPath + ". This probably " +
|
||||
"indicates that you either have configured a brokerid that is already in use, or " +
|
||||
"else you have shutdown this broker and restarted it faster than the zookeeper " +
|
||||
"timeout so it appears to be re-registering.")
|
||||
throw new RuntimeException("A broker is already registered on the path " + brokerIdPath + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + "else you have shutdown this broker and restarted it faster than the zookeeper " + "timeout so it appears to be re-registering.")
|
||||
}
|
||||
info("Registering broker " + brokerIdPath + " succeeded with " + broker)
|
||||
}
|
||||
|
||||
def getConsumerPartitionOwnerPath(group: String, topic: String, partition: String): String = {
|
||||
def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = {
|
||||
val topicDirs = new ZKGroupTopicDirs(group, topic)
|
||||
topicDirs.consumerOwnerDir + "/" + partition
|
||||
}
|
||||
|
@ -254,7 +252,7 @@ object ZkUtils extends Logging {
|
|||
// this can happen when there is connection loss; make sure the data is what we intend to write
|
||||
var storedData: String = null
|
||||
try {
|
||||
storedData = readData(client, path)
|
||||
storedData = readData(client, path)._1
|
||||
} catch {
|
||||
case e1: ZkNoNodeException => // the node disappeared; treat as if node existed and let caller handles this
|
||||
case e2 => throw e2
|
||||
|
@ -292,17 +290,24 @@ object ZkUtils extends Logging {
|
|||
/**
|
||||
* Update the value of a persistent node with the given path and data.
|
||||
* create parrent directory if necessary. Never throw NodeExistException.
|
||||
* Return the updated path zkVersion
|
||||
*/
|
||||
def updatePersistentPath(client: ZkClient, path: String, data: String): Unit = {
|
||||
def updatePersistentPath(client: ZkClient, path: String, data: String): Int = {
|
||||
var stat: Stat = null
|
||||
try {
|
||||
client.writeData(path, data)
|
||||
stat = client.writeData(path, data)
|
||||
return stat.getVersion
|
||||
} catch {
|
||||
case e: ZkNoNodeException => {
|
||||
createParentPath(client, path)
|
||||
try {
|
||||
client.createPersistent(path, data)
|
||||
// When the new path is created, its zkVersion always starts from 0
|
||||
return 0
|
||||
} catch {
|
||||
case e: ZkNodeExistsException => client.writeData(path, data)
|
||||
case e: ZkNodeExistsException =>
|
||||
stat = client.writeData(path, data)
|
||||
return stat.getVersion
|
||||
case e2 => throw e2
|
||||
}
|
||||
}
|
||||
|
@ -310,6 +315,22 @@ object ZkUtils extends Logging {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't
|
||||
* exist, the current version is not the expected version, etc.) return (false, -1)
|
||||
*/
|
||||
def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = {
|
||||
try {
|
||||
val stat = client.writeData(path, data, expectVersion)
|
||||
info("Conditional update the zkPath %s with expected version %d succeed and return the new version: %d".format(path, expectVersion, stat.getVersion))
|
||||
(true, stat.getVersion)
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
info("Conditional update the zkPath %s with expected version %d failed".format(path, expectVersion))
|
||||
(false, -1)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Update the value of a persistent node with the given path and data.
|
||||
* create parrent directory if necessary. Never throw NodeExistException.
|
||||
|
@ -349,12 +370,23 @@ object ZkUtils extends Logging {
|
|||
}
|
||||
}
|
||||
|
||||
def readData(client: ZkClient, path: String): String = {
|
||||
client.readData(path)
|
||||
def readData(client: ZkClient, path: String): (String, Stat) = {
|
||||
val stat: Stat = new Stat()
|
||||
val dataStr: String = client.readData(path, stat)
|
||||
(dataStr, stat)
|
||||
}
|
||||
|
||||
def readDataMaybeNull(client: ZkClient, path: String): String = {
|
||||
client.readData(path, true)
|
||||
def readDataMaybeNull(client: ZkClient, path: String): (String, Stat) = {
|
||||
val stat: Stat = new Stat()
|
||||
var dataStr: String = null
|
||||
try{
|
||||
dataStr = client.readData(path, stat)
|
||||
return (dataStr, stat)
|
||||
} catch {
|
||||
case e: ZkNoNodeException =>
|
||||
return (null, stat)
|
||||
case e2 => throw e2
|
||||
}
|
||||
}
|
||||
|
||||
def getChildren(client: ZkClient, path: String): Seq[String] = {
|
||||
|
@ -366,7 +398,6 @@ object ZkUtils extends Logging {
|
|||
def getChildrenParentMayNotExist(client: ZkClient, path: String): Seq[String] = {
|
||||
import scala.collection.JavaConversions._
|
||||
// triggers implicit conversion from java list to scala Seq
|
||||
|
||||
try {
|
||||
client.getChildren(path)
|
||||
} catch {
|
||||
|
@ -388,32 +419,77 @@ object ZkUtils extends Logging {
|
|||
val cluster = new Cluster
|
||||
val nodes = getChildrenParentMayNotExist(zkClient, BrokerIdsPath)
|
||||
for (node <- nodes) {
|
||||
val brokerZKString = readData(zkClient, BrokerIdsPath + "/" + node)
|
||||
val brokerZKString = readData(zkClient, BrokerIdsPath + "/" + node)._1
|
||||
cluster.add(Broker.createBroker(node.toInt, brokerZKString))
|
||||
}
|
||||
cluster
|
||||
}
|
||||
|
||||
def getPartitionAssignmentForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, Map[String, List[String]]] = {
|
||||
val ret = new mutable.HashMap[String, Map[String, List[String]]]()
|
||||
def getReplicaAssignmentForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[(String, Int), Seq[Int]] = {
|
||||
val ret = new mutable.HashMap[(String, Int), Seq[Int]]
|
||||
topics.foreach{ topic =>
|
||||
val jsonPartitionMap = readDataMaybeNull(zkClient, getTopicPath(topic))
|
||||
val partitionMap = if (jsonPartitionMap == null) {
|
||||
Map[String, List[String]]()
|
||||
} else {
|
||||
val jsonPartitionMap = readDataMaybeNull(zkClient, getTopicPath(topic))._1
|
||||
if (jsonPartitionMap != null) {
|
||||
JSON.parseFull(jsonPartitionMap) match {
|
||||
case Some(m) => m.asInstanceOf[Map[String, List[String]]]
|
||||
case None => Map[String, List[String]]()
|
||||
case Some(m) =>
|
||||
val replicaMap = m.asInstanceOf[Map[String, Seq[String]]]
|
||||
for((partition, replicas) <- replicaMap){
|
||||
ret.put((topic, partition.toInt), replicas.map(_.toInt))
|
||||
debug("Replicas assigned to topic [%s], partition [%s] are [%s]".format(topic, partition, replicas))
|
||||
}
|
||||
case None =>
|
||||
}
|
||||
}
|
||||
debug("partition map for /brokers/topics/%s is %s".format(topic, partitionMap))
|
||||
ret += (topic -> partitionMap)
|
||||
}
|
||||
ret
|
||||
}
|
||||
|
||||
def getPartitionLeaderAndISRForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[(String, Int), LeaderAndISR] = {
|
||||
val ret = new mutable.HashMap[(String, Int), LeaderAndISR]
|
||||
val partitionsForTopics = getPartitionsForTopics(zkClient, topics)
|
||||
for((topic, partitions) <- partitionsForTopics){
|
||||
for(partition <- partitions){
|
||||
val leaderAndISROpt = ZkUtils.getLeaderAndISRForPartition(zkClient, topic, partition.toInt)
|
||||
if(leaderAndISROpt.isDefined)
|
||||
ret.put((topic, partition.toInt), leaderAndISROpt.get)
|
||||
}
|
||||
}
|
||||
ret
|
||||
}
|
||||
|
||||
def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, Seq[String]] = {
|
||||
getPartitionAssignmentForTopics(zkClient, topics).map{ topicAndPartitionMap =>
|
||||
def getPartitionAssignmentForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, collection.Map[Int, Seq[Int]]] = {
|
||||
val ret = new mutable.HashMap[String, Map[Int, Seq[Int]]]()
|
||||
topics.foreach{ topic =>
|
||||
val jsonPartitionMap = readDataMaybeNull(zkClient, getTopicPath(topic))._1
|
||||
val partitionMap = if (jsonPartitionMap == null) {
|
||||
Map[Int, Seq[Int]]()
|
||||
} else {
|
||||
JSON.parseFull(jsonPartitionMap) match {
|
||||
case Some(m) =>
|
||||
val m1 = m.asInstanceOf[Map[String, Seq[String]]]
|
||||
m1.map(p => (p._1.toInt, p._2.map(_.toInt)))
|
||||
case None => Map[Int, Seq[Int]]()
|
||||
}
|
||||
}
|
||||
debug("partition map for /brokers/topics/%s is %s".format(topic, partitionMap))
|
||||
ret += (topic -> partitionMap)
|
||||
}
|
||||
ret
|
||||
}
|
||||
|
||||
def getReplicaAssignmentFromPartitionAssignment(topicPartitionAssignment: mutable.Map[String, collection.Map[Int, Seq[Int]]]): mutable.Map[(String, Int), Seq[Int]] = {
|
||||
val ret = new mutable.HashMap[(String, Int), Seq[Int]]
|
||||
for((topic, partitionAssignment) <- topicPartitionAssignment){
|
||||
for((partition, replicaAssignment) <- partitionAssignment){
|
||||
ret.put((topic, partition), replicaAssignment)
|
||||
}
|
||||
}
|
||||
ret
|
||||
}
|
||||
|
||||
def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, Seq[Int]] = {
|
||||
getPartitionAssignmentForTopics(zkClient, topics).map
|
||||
{ topicAndPartitionMap =>
|
||||
val topic = topicAndPartitionMap._1
|
||||
val partitionMap = topicAndPartitionMap._2
|
||||
debug("partition assignment of /brokers/topics/%s is %s".format(topic, partitionMap))
|
||||
|
@ -421,14 +497,20 @@ object ZkUtils extends Logging {
|
|||
}
|
||||
}
|
||||
|
||||
def getPartitionsAssignedToBroker(zkClient: ZkClient, topics: Seq[String], brokerId: Int): Map[String, Seq[Int]] = {
|
||||
def getPartitionsAssignedToBroker(zkClient: ZkClient, topics: Seq[String], brokerId: Int): Map[(String, Int), Seq[Int]] = {
|
||||
val ret = new mutable.HashMap[(String, Int), Seq[Int]]
|
||||
val topicsAndPartitions = getPartitionAssignmentForTopics(zkClient, topics.iterator)
|
||||
topicsAndPartitions.map{ topicAndPartitionMap =>
|
||||
val topic = topicAndPartitionMap._1
|
||||
val partitionMap = topicAndPartitionMap._2
|
||||
val relevantPartitions = partitionMap.filter( m => m._2.contains(brokerId.toString) )
|
||||
(topic -> relevantPartitions.keySet.map(_.toInt).toSeq)
|
||||
topicsAndPartitions.map
|
||||
{
|
||||
topicAndPartitionMap =>
|
||||
val topic = topicAndPartitionMap._1
|
||||
val partitionMap = topicAndPartitionMap._2
|
||||
val relevantPartitionsMap = partitionMap.filter( m => m._2.contains(brokerId) )
|
||||
for((relevantPartition, replicaAssignment) <- relevantPartitionsMap){
|
||||
ret.put((topic, relevantPartition), replicaAssignment)
|
||||
}
|
||||
}
|
||||
ret
|
||||
}
|
||||
|
||||
def deletePartition(zkClient : ZkClient, brokerId: Int, topic: String) {
|
||||
|
@ -446,8 +528,7 @@ object ZkUtils extends Logging {
|
|||
def getConsumerTopicMaps(zkClient: ZkClient, group: String): Map[String, TopicCount] = {
|
||||
val dirs = new ZKGroupDirs(group)
|
||||
val consumersInGroup = getConsumersInGroup(zkClient, group)
|
||||
val topicCountMaps = consumersInGroup.map(consumerId => TopicCount.constructTopicCount(consumerId,
|
||||
ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId), zkClient))
|
||||
val topicCountMaps = consumersInGroup.map(consumerId => TopicCount.constructTopicCount(consumerId,ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1, zkClient))
|
||||
consumersInGroup.zip(topicCountMaps).toMap
|
||||
}
|
||||
|
||||
|
@ -470,8 +551,7 @@ object ZkUtils extends Logging {
|
|||
consumersPerTopicMap
|
||||
}
|
||||
|
||||
def getBrokerInfoFromIds(zkClient: ZkClient, brokerIds: Seq[Int]): Seq[Broker] =
|
||||
brokerIds.map( bid => Broker.createBroker(bid, ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + bid)) )
|
||||
def getBrokerInfoFromIds(zkClient: ZkClient, brokerIds: Seq[Int]): Seq[Broker] = brokerIds.map( bid => Broker.createBroker(bid, ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + bid)._1))
|
||||
|
||||
def getAllTopics(zkClient: ZkClient): Seq[String] = {
|
||||
val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
|
||||
|
@ -479,17 +559,45 @@ object ZkUtils extends Logging {
|
|||
else topics
|
||||
}
|
||||
|
||||
def incrementEpochForPartition(client: ZkClient, topic: String, partition: Int) = {
|
||||
// read previous epoch, increment it and write it to the leader path and the ISR path.
|
||||
val epoch = try {
|
||||
Some(getEpochForPartition(client, topic, partition))
|
||||
}catch {
|
||||
case e: NoEpochForPartitionException => None
|
||||
case e1 => throw e1
|
||||
}
|
||||
val newEpoch = epoch match {
|
||||
case Some(partitionEpoch) =>
|
||||
debug("Existing epoch for topic %s partition %d is %d".format(topic, partition, partitionEpoch))
|
||||
partitionEpoch + 1
|
||||
case None =>
|
||||
// this is the first time leader is elected for this partition. So set epoch to 1
|
||||
debug("First epoch is 1 for topic %s partition %d".format(topic, partition))
|
||||
LeaderAndISR.initialLeaderEpoch
|
||||
}
|
||||
newEpoch
|
||||
}
|
||||
}
|
||||
|
||||
class LeaderExistsListener(topic: String, partition: Int, leaderLock: ReentrantLock, leaderExists: Condition) extends IZkDataListener {
|
||||
class LeaderElectionListener(topic: String,
|
||||
partition: Int,
|
||||
leaderLock: ReentrantLock,
|
||||
leaderExistsOrChanged: Condition,
|
||||
zkClient: ZkClient = null) extends IZkDataListener with Logging {
|
||||
@throws(classOf[Exception])
|
||||
def handleDataChange(dataPath: String, data: Object) {
|
||||
val t = dataPath.split("/").takeRight(3).head
|
||||
val p = dataPath.split("/").takeRight(2).head.toInt
|
||||
leaderLock.lock()
|
||||
try {
|
||||
if(t == topic && p == partition)
|
||||
leaderExists.signal()
|
||||
if(t == topic && p == partition){
|
||||
val newLeaderOpt = ZkUtils.getLeaderForPartition(zkClient, t, p)
|
||||
if(newLeaderOpt.isDefined && ZkUtils.pathExists (zkClient, ZkUtils.getBrokerPath(newLeaderOpt.get))){
|
||||
trace("In leader election listener on partition [%s, %d], live leader %d is elected".format(topic, partition, newLeaderOpt.get))
|
||||
leaderExistsOrChanged.signal()
|
||||
}
|
||||
}
|
||||
}
|
||||
finally {
|
||||
leaderLock.unlock()
|
||||
|
@ -497,15 +605,9 @@ class LeaderExistsListener(topic: String, partition: Int, leaderLock: ReentrantL
|
|||
}
|
||||
|
||||
@throws(classOf[Exception])
|
||||
def handleDataDeleted(dataPath: String) {
|
||||
leaderLock.lock()
|
||||
try {
|
||||
leaderExists.signal()
|
||||
}finally {
|
||||
leaderLock.unlock()
|
||||
}
|
||||
def handleDataDeleted(dataPath: String){
|
||||
// Nothing
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object ZKStringSerializer extends ZkSerializer {
|
||||
|
|
|
@ -95,8 +95,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
val sentMessages1 = (sentMessages1_1 ++ sentMessages1_2).sortWith((s,t) => s.checksum < t.checksum)
|
||||
|
||||
// wait to make sure the topic and partition have a leader for the successful case
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
// create a consumer
|
||||
val consumerConfig1 = new ConsumerConfig(
|
||||
|
@ -128,8 +128,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
val sentMessages2_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages)
|
||||
val sentMessages2 = (sentMessages2_1 ++ sentMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1)
|
||||
val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2)
|
||||
|
@ -153,8 +153,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
val sentMessages3_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages)
|
||||
val sentMessages3 = (sentMessages3_1 ++ sentMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1)
|
||||
val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2)
|
||||
|
@ -182,8 +182,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
val sentMessages1_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
|
||||
val sentMessages1 = (sentMessages1_1 ++ sentMessages1_2).sortWith((s,t) => s.checksum < t.checksum)
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
// create a consumer
|
||||
val consumerConfig1 = new ConsumerConfig(
|
||||
|
@ -214,8 +214,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
val sentMessages2_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
|
||||
val sentMessages2 = (sentMessages2_1 ++ sentMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1)
|
||||
val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2)
|
||||
|
@ -239,8 +239,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
val sentMessages3_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
|
||||
val sentMessages3 = (sentMessages3_1 ++ sentMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1)
|
||||
val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2)
|
||||
|
@ -320,8 +320,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
|
||||
val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1))
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
val zkConsumerConnector =
|
||||
new ZookeeperConsumerConnector(consumerConfig, true)
|
||||
|
|
|
@ -25,7 +25,8 @@ import kafka.server.{KafkaServer, KafkaConfig}
|
|||
import kafka.api._
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import kafka.utils.{ControllerTestUtils, ZkUtils, TestUtils}
|
||||
import kafka.admin.CreateTopicCommand
|
||||
import kafka.utils.{ZkUtils, ControllerTestUtils, TestUtils}
|
||||
|
||||
|
||||
class ControllerBasicTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||
|
@ -36,11 +37,13 @@ class ControllerBasicTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
override def setUp() {
|
||||
super.setUp()
|
||||
brokers = configs.map(config => TestUtils.createServer(config))
|
||||
CreateTopicCommand.createTopic(zkClient, "test1", 1, 4, "0:1:2:3")
|
||||
CreateTopicCommand.createTopic(zkClient, "test2", 1, 4, "0:1:2:3")
|
||||
}
|
||||
|
||||
override def tearDown() {
|
||||
super.tearDown()
|
||||
brokers.foreach(_.shutdown())
|
||||
super.tearDown()
|
||||
}
|
||||
|
||||
def testControllerFailOver(){
|
||||
|
@ -49,35 +52,39 @@ class ControllerBasicTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
brokers(3).shutdown()
|
||||
Thread.sleep(1000)
|
||||
|
||||
var curController = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.ControllerPath)
|
||||
var curController = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.ControllerPath)._1
|
||||
info("cur controller " + curController)
|
||||
assertEquals(curController, "2")
|
||||
|
||||
|
||||
brokers(1).startup()
|
||||
brokers(2).shutdown()
|
||||
Thread.sleep(1000)
|
||||
curController = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.ControllerPath)
|
||||
assertEquals(curController, "1")
|
||||
curController = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.ControllerPath)._1
|
||||
info("cur controller " + curController)
|
||||
assertEquals("Controller should be on broker 1", curController, "1")
|
||||
}
|
||||
|
||||
def testControllerCommandSend(){
|
||||
Thread.sleep(1000)
|
||||
|
||||
for(broker <- brokers){
|
||||
if(broker.kafkaController.isActive){
|
||||
val leaderAndISRRequest = ControllerTestUtils.createSampleLeaderAndISRRequest()
|
||||
val stopReplicaRequest = ControllerTestUtils.createSampleStopReplicaRequest()
|
||||
val leaderAndISRRequest = ControllerTestUtils.createTestLeaderAndISRRequest()
|
||||
val stopReplicaRequest = ControllerTestUtils.createTestStopReplicaRequest()
|
||||
|
||||
val successCount: AtomicInteger = new AtomicInteger(0)
|
||||
val countDownLatch: CountDownLatch = new CountDownLatch(8)
|
||||
|
||||
def compareLeaderAndISRResponseWithExpectedOne(response: RequestOrResponse){
|
||||
val expectedResponse = ControllerTestUtils.createSampleLeaderAndISRResponse()
|
||||
val expectedResponse = ControllerTestUtils.createTestLeaderAndISRResponse()
|
||||
if(response.equals(expectedResponse))
|
||||
successCount.addAndGet(1)
|
||||
countDownLatch.countDown()
|
||||
}
|
||||
|
||||
def compareStopReplicaResponseWithExpectedOne(response: RequestOrResponse){
|
||||
val expectedResponse = ControllerTestUtils.createSampleStopReplicaResponse()
|
||||
val expectedResponse = ControllerTestUtils.createTestStopReplicaResponse()
|
||||
if(response.equals(expectedResponse))
|
||||
successCount.addAndGet(1)
|
||||
countDownLatch.countDown()
|
||||
|
@ -87,10 +94,10 @@ class ControllerBasicTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
broker.kafkaController.sendRequest(1, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(2, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(3, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(0, stopReplicaRequest, compareStopReplicaResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(1, stopReplicaRequest, compareStopReplicaResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(2, stopReplicaRequest, compareStopReplicaResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(3, stopReplicaRequest, compareStopReplicaResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(0, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(1, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(2, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
broker.kafkaController.sendRequest(3, leaderAndISRRequest, compareLeaderAndISRResponseWithExpectedOne)
|
||||
countDownLatch.await()
|
||||
|
||||
assertEquals(successCount.get(), 8)
|
||||
|
|
|
@ -1,173 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.controller
|
||||
|
||||
|
||||
import org.scalatest.junit.JUnit3Suite
|
||||
|
||||
import junit.framework.Assert._
|
||||
import java.nio.ByteBuffer
|
||||
import kafka.common.ErrorMapping
|
||||
import kafka.api._
|
||||
import collection.mutable.Map
|
||||
import collection.mutable.Set
|
||||
import kafka.integration.KafkaServerTestHarness
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.network.{Receive, BlockingChannel}
|
||||
|
||||
|
||||
class ControllerToBrokerRequestTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||
|
||||
val kafkaProps = TestUtils.createBrokerConfigs(1)
|
||||
val configs = List(new KafkaConfig(kafkaProps.head))
|
||||
var blockingChannel: BlockingChannel = null
|
||||
|
||||
override def setUp() {
|
||||
super.setUp()
|
||||
blockingChannel = new BlockingChannel("localhost", configs.head.port, 1000000, 0, 64*1024)
|
||||
blockingChannel.connect
|
||||
}
|
||||
|
||||
override def tearDown() {
|
||||
super.tearDown()
|
||||
blockingChannel.disconnect()
|
||||
}
|
||||
|
||||
|
||||
def createSampleLeaderAndISRRequest() : LeaderAndISRRequest = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
|
||||
val leader1 = 1;
|
||||
val ISR1 = List(1, 2, 3)
|
||||
|
||||
val leader2 = 2;
|
||||
val ISR2 = List(2, 3, 4)
|
||||
|
||||
val leaderAndISR1 = new LeaderAndISR(leader1, 1, ISR1, 1)
|
||||
val leaderAndISR2 = new LeaderAndISR(leader2, 1, ISR2, 2)
|
||||
val map = Map(((topic1, 1), leaderAndISR1), ((topic1, 2), leaderAndISR1),
|
||||
((topic2, 1), leaderAndISR2), ((topic2, 2), leaderAndISR2))
|
||||
|
||||
new LeaderAndISRRequest(1, "client 1", 1, 4, map)
|
||||
}
|
||||
|
||||
def createSampleLeaderAndISRResponse() : LeaderAndISRResponse = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
val responseMap = Map(((topic1, 1), ErrorMapping.NoError), ((topic1, 2), ErrorMapping.NoError),
|
||||
((topic2, 1), ErrorMapping.NoError), ((topic2, 2), ErrorMapping.NoError))
|
||||
|
||||
new LeaderAndISRResponse(1, responseMap)
|
||||
}
|
||||
|
||||
|
||||
def createSampleStopReplicaRequest() : StopReplicaRequest = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
new StopReplicaRequest(1, "client 1", 1000, Set((topic1, 1), (topic1, 2),
|
||||
(topic2, 1), (topic2, 2)))
|
||||
}
|
||||
|
||||
def createSampleStopReplicaResponse() : StopReplicaResponse = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
val responseMap = Map(((topic1, 1), ErrorMapping.NoError), ((topic1, 2), ErrorMapping.NoError),
|
||||
((topic2, 1), ErrorMapping.NoError), ((topic2, 2), ErrorMapping.NoError))
|
||||
|
||||
new StopReplicaResponse(1, responseMap)
|
||||
}
|
||||
|
||||
|
||||
def testLeaderAndISRRequest {
|
||||
val leaderAndISRRequest = createSampleLeaderAndISRRequest()
|
||||
|
||||
val serializedLeaderAndISRRequest = ByteBuffer.allocate(leaderAndISRRequest.sizeInBytes)
|
||||
leaderAndISRRequest.writeTo(serializedLeaderAndISRRequest)
|
||||
serializedLeaderAndISRRequest.rewind()
|
||||
val deserializedLeaderAndISRRequest = LeaderAndISRRequest.readFrom(serializedLeaderAndISRRequest)
|
||||
|
||||
assertEquals(leaderAndISRRequest, deserializedLeaderAndISRRequest)
|
||||
}
|
||||
|
||||
def testLeaderAndISRResponse {
|
||||
val leaderAndISRResponse = createSampleLeaderAndISRResponse()
|
||||
|
||||
val serializedLeaderAndISRResponse = ByteBuffer.allocate(leaderAndISRResponse.sizeInBytes)
|
||||
leaderAndISRResponse.writeTo(serializedLeaderAndISRResponse)
|
||||
serializedLeaderAndISRResponse.rewind()
|
||||
val deserializedLeaderAndISRResponse = LeaderAndISRResponse.readFrom(serializedLeaderAndISRResponse)
|
||||
assertEquals(leaderAndISRResponse, deserializedLeaderAndISRResponse)
|
||||
}
|
||||
|
||||
|
||||
def testStopReplicaRequest {
|
||||
val stopReplicaRequest = createSampleStopReplicaRequest()
|
||||
|
||||
val serializedStopReplicaRequest = ByteBuffer.allocate(stopReplicaRequest.sizeInBytes)
|
||||
stopReplicaRequest.writeTo(serializedStopReplicaRequest)
|
||||
serializedStopReplicaRequest.rewind()
|
||||
val deserializedStopReplicaRequest = StopReplicaRequest.readFrom(serializedStopReplicaRequest)
|
||||
assertEquals(stopReplicaRequest, deserializedStopReplicaRequest)
|
||||
}
|
||||
|
||||
|
||||
def testStopReplicaResponse {
|
||||
val stopReplicaResponse = createSampleStopReplicaResponse()
|
||||
|
||||
val serializedStopReplicaResponse = ByteBuffer.allocate(stopReplicaResponse.sizeInBytes)
|
||||
stopReplicaResponse.writeTo(serializedStopReplicaResponse)
|
||||
serializedStopReplicaResponse.rewind()
|
||||
val deserializedStopReplicaResponse = StopReplicaResponse.readFrom(serializedStopReplicaResponse)
|
||||
assertEquals(stopReplicaResponse, deserializedStopReplicaResponse)
|
||||
}
|
||||
|
||||
|
||||
|
||||
def testEndToEndLeaderAndISRRequest {
|
||||
|
||||
val leaderAndISRRequest = createSampleLeaderAndISRRequest()
|
||||
|
||||
var response: Receive = null
|
||||
blockingChannel.send(leaderAndISRRequest)
|
||||
response = blockingChannel.receive()
|
||||
|
||||
val leaderAndISRResponse = LeaderAndISRResponse.readFrom(response.buffer)
|
||||
val expectedLeaderAndISRResponse = createSampleLeaderAndISRResponse()
|
||||
|
||||
assertEquals(leaderAndISRResponse, expectedLeaderAndISRResponse)
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
def testEndToEndStopReplicaRequest {
|
||||
val stopReplicaRequest = createSampleStopReplicaRequest()
|
||||
|
||||
var response: Receive = null
|
||||
blockingChannel.send(stopReplicaRequest)
|
||||
response = blockingChannel.receive()
|
||||
|
||||
val stopReplicaResponse = StopReplicaResponse.readFrom(response.buffer)
|
||||
val expectedStopReplicaResponse = createSampleStopReplicaResponse()
|
||||
assertEquals(stopReplicaResponse, expectedStopReplicaResponse)
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -57,7 +57,7 @@ class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness
|
|||
// test for reading data with magic byte 0
|
||||
def testProtocolVersion0() {
|
||||
CreateTopicCommand.createTopic(zkClient, topic, 0, 1, configs.head.brokerId.toString)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
val lastOffset = simpleConsumer.getOffsetsBefore(topic, 0, OffsetRequest.LatestTime, 1)
|
||||
var fetchOffset: Long = 0L
|
||||
var messageCount: Int = 0
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -35,27 +35,28 @@ import kafka.admin.CreateTopicCommand
|
|||
class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
||||
|
||||
val numNodes = 1
|
||||
val configs =
|
||||
val configs =
|
||||
for(props <- TestUtils.createBrokerConfigs(numNodes))
|
||||
yield new KafkaConfig(props)
|
||||
yield new KafkaConfig(props)
|
||||
val messages = new mutable.HashMap[Int, Seq[Message]]
|
||||
val topic = "topic"
|
||||
val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port)))
|
||||
val shutdown = ZookeeperConsumerConnector.shutdownCommand
|
||||
val queue = new LinkedBlockingQueue[FetchedDataChunk]
|
||||
val topicInfos = configs.map(c => new PartitionTopicInfo(topic,
|
||||
c.brokerId,
|
||||
0,
|
||||
queue,
|
||||
new AtomicLong(0),
|
||||
new AtomicLong(0),
|
||||
new AtomicInteger(0)))
|
||||
|
||||
c.brokerId,
|
||||
0,
|
||||
queue,
|
||||
new AtomicLong(0),
|
||||
new AtomicLong(0),
|
||||
new AtomicInteger(0)))
|
||||
|
||||
var fetcher: ConsumerFetcherManager = null
|
||||
|
||||
override def setUp() {
|
||||
super.setUp
|
||||
CreateTopicCommand.createTopic(zkClient, topic, 1, 1, configs.head.brokerId.toString)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient)
|
||||
fetcher.stopAllConnections()
|
||||
fetcher.startConnections(topicInfos, cluster)
|
||||
|
@ -65,20 +66,20 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
|||
fetcher.shutdown()
|
||||
super.tearDown
|
||||
}
|
||||
|
||||
|
||||
def testFetcher() {
|
||||
val perNode = 2
|
||||
var count = sendMessages(perNode)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
|
||||
fetch(count)
|
||||
assertQueueEmpty()
|
||||
count = sendMessages(perNode)
|
||||
fetch(count)
|
||||
assertQueueEmpty()
|
||||
}
|
||||
|
||||
|
||||
def assertQueueEmpty(): Unit = assertEquals(0, queue.size)
|
||||
|
||||
|
||||
def sendMessages(messagesPerNode: Int): Int = {
|
||||
var count = 0
|
||||
for(conf <- configs) {
|
||||
|
@ -91,7 +92,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
|||
}
|
||||
count
|
||||
}
|
||||
|
||||
|
||||
def fetch(expected: Int) {
|
||||
var count = 0
|
||||
while(true) {
|
||||
|
@ -103,5 +104,5 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
|
|||
return
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -134,7 +134,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
|
|||
builder.addFetch(topic, 0, 0, 10000)
|
||||
}
|
||||
// wait until leader is elected
|
||||
topics.foreach(topic => TestUtils.waitUntilLeaderIsElected(zkClient, topic, 0, 500))
|
||||
topics.foreach(topic => TestUtils.waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500))
|
||||
producer.send(produceList: _*)
|
||||
|
||||
// wait a bit for produced message to be available
|
||||
|
@ -160,7 +160,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
|
|||
builder.addFetch(topic, 0, 0, 10000)
|
||||
}
|
||||
// wait until leader is elected
|
||||
topics.foreach(topic => TestUtils.waitUntilLeaderIsElected(zkClient, topic, 0, 1500))
|
||||
topics.foreach(topic => TestUtils.waitUntilLiveLeaderIsElected(zkClient, topic, 0, 1500))
|
||||
|
||||
producer.send(produceList: _*)
|
||||
|
||||
|
|
|
@ -352,7 +352,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
|
|||
def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Seq[String], brokerId: Int) {
|
||||
for( topic <- topics ) {
|
||||
CreateTopicCommand.createTopic(zkClient, topic, 1, 1, brokerId.toString)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,8 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
// create the kafka request handler
|
||||
val requestChannel = new RequestChannel(2, 5)
|
||||
val apis = new KafkaApis(requestChannel, logManager, replicaManager, kafkaZookeeper)
|
||||
val apis = new KafkaApis(requestChannel, logManager, replicaManager, kafkaZookeeper, null,
|
||||
null, null, null, 1)
|
||||
|
||||
// mock the receive API to return the request buffer as created above
|
||||
val receivedRequest = EasyMock.createMock(classOf[BoundedByteBufferReceive])
|
||||
|
|
|
@ -54,8 +54,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar
|
|||
// send some messages to each broker
|
||||
val sentMessages1 = sendMessages(nMessages, "batch1")
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLeaderIsElected(zkClient, topic, 1, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 1, 500)
|
||||
|
||||
// create a consumer
|
||||
val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zookeeperConnect, group, consumer1))
|
||||
|
|
|
@ -118,7 +118,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
// setup brokers in zookeeper as owners of partitions for this test
|
||||
CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "1")
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, topic, 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, topic, 0, 500)
|
||||
|
||||
var offsetChanged = false
|
||||
for(i <- 1 to 14) {
|
||||
|
|
|
@ -27,9 +27,10 @@ import junit.framework.Assert._
|
|||
|
||||
class SocketServerTest extends JUnitSuite {
|
||||
|
||||
val server: SocketServer = new SocketServer(port = TestUtils.choosePort,
|
||||
numProcessorThreads = 1,
|
||||
monitoringPeriodSecs = 30,
|
||||
val server: SocketServer = new SocketServer(0,
|
||||
port = TestUtils.choosePort,
|
||||
numProcessorThreads = 1,
|
||||
monitoringPeriodSecs = 30,
|
||||
maxQueuedRequests = 50,
|
||||
maxRequestSize = 50)
|
||||
server.startup()
|
||||
|
|
|
@ -106,13 +106,6 @@ class AsyncProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
}
|
||||
}
|
||||
|
||||
def getProduceData(nEvents: Int): Seq[ProducerData[String,String]] = {
|
||||
val producerDataList = new ListBuffer[ProducerData[String,String]]
|
||||
for (i <- 0 until nEvents)
|
||||
producerDataList.append(new ProducerData[String,String]("topic1", null, List("msg" + i)))
|
||||
producerDataList
|
||||
}
|
||||
|
||||
@Test
|
||||
def testBatchSize() {
|
||||
/**
|
||||
|
@ -530,6 +523,13 @@ class AsyncProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
}
|
||||
}
|
||||
|
||||
def getProduceData(nEvents: Int): Seq[ProducerData[String,String]] = {
|
||||
val producerDataList = new ListBuffer[ProducerData[String,String]]
|
||||
for (i <- 0 until nEvents)
|
||||
producerDataList.append(new ProducerData[String,String]("topic1", null, List("msg" + i)))
|
||||
producerDataList
|
||||
}
|
||||
|
||||
private def messagesToSet(messages: Seq[String]): ByteBufferMessageSet = {
|
||||
val encoder = new StringEncoder
|
||||
new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => encoder.toMessage(m)): _*)
|
||||
|
|
|
@ -104,7 +104,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
// create topic with 1 partition and await leadership
|
||||
CreateTopicCommand.createTopic(zkClient, "new-topic", 1, 2)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "new-topic", 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "new-topic", 0, 500)
|
||||
|
||||
val producer1 = new Producer[String, String](config1)
|
||||
val producer2 = new Producer[String, String](config2)
|
||||
|
@ -155,10 +155,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
// create topic
|
||||
CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0,0,0,0")
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "new-topic", 0, 500)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "new-topic", 1, 500)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "new-topic", 2, 500)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "new-topic", 3, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "new-topic", 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "new-topic", 1, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "new-topic", 2, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "new-topic", 3, 500)
|
||||
|
||||
val config = new ProducerConfig(props)
|
||||
val producer = new Producer[String, String](config)
|
||||
|
@ -216,7 +216,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
// create topics in ZK
|
||||
CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0:1,0:1,0:1,0:1")
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "new-topic", 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "new-topic", 0, 500)
|
||||
|
||||
// do a simple test to make sure plumbing is okay
|
||||
try {
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -159,9 +159,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
|
|||
|
||||
// #2 - test that we get correct offsets when partition is owned by broker
|
||||
CreateTopicCommand.createTopic(zkClient, "topic1", 1, 1)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "topic1", 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "topic1", 0, 500)
|
||||
CreateTopicCommand.createTopic(zkClient, "topic3", 1, 1)
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, "topic3", 0, 500)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, "topic3", 0, 500)
|
||||
|
||||
val response2 = producer.send(request)
|
||||
Assert.assertNotNull(response2)
|
||||
|
|
|
@ -39,8 +39,9 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
|
|||
val scheduler = new KafkaScheduler(2)
|
||||
scheduler.startUp
|
||||
// create replica manager
|
||||
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler)
|
||||
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, null)
|
||||
replicaManager.startup()
|
||||
replicaManager.startHighWaterMarksCheckPointThread()
|
||||
// sleep until flush ms
|
||||
Thread.sleep(configs.head.defaultFlushIntervalMs)
|
||||
var fooPartition0Hw = replicaManager.readCheckpointedHighWatermark(topic, 0)
|
||||
|
@ -83,8 +84,9 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
|
|||
val scheduler = new KafkaScheduler(2)
|
||||
scheduler.startUp
|
||||
// create replica manager
|
||||
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler)
|
||||
val replicaManager = new ReplicaManager(configs.head, new MockTime(), zkClient, scheduler, null)
|
||||
replicaManager.startup()
|
||||
replicaManager.startHighWaterMarksCheckPointThread()
|
||||
// sleep until flush ms
|
||||
Thread.sleep(configs.head.defaultFlushIntervalMs)
|
||||
var topic1Partition0Hw = replicaManager.readCheckpointedHighWatermark(topic1, 0)
|
||||
|
|
|
@ -100,7 +100,7 @@ class ISRExpirationTest extends JUnit3Suite {
|
|||
// create kafka scheduler
|
||||
val scheduler = new KafkaScheduler(2)
|
||||
// create replica manager
|
||||
val replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler)
|
||||
val replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler, null)
|
||||
try {
|
||||
val partition0 = replicaManager.getOrCreatePartition(topic, 0, configs.map(_.brokerId).toSet)
|
||||
// create leader log
|
||||
|
|
|
@ -20,12 +20,11 @@ package kafka.server
|
|||
import org.scalatest.junit.JUnit3Suite
|
||||
import kafka.zk.ZooKeeperTestHarness
|
||||
import kafka.admin.CreateTopicCommand
|
||||
import kafka.utils.TestUtils._
|
||||
import junit.framework.Assert._
|
||||
import kafka.utils.{ZkUtils, Utils, TestUtils}
|
||||
import kafka.utils.TestUtils._
|
||||
|
||||
class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||
|
||||
val brokerId1 = 0
|
||||
val brokerId2 = 1
|
||||
|
||||
|
@ -34,23 +33,23 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1)
|
||||
val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2)
|
||||
|
||||
var servers: Seq[KafkaServer] = Seq.empty[KafkaServer]
|
||||
|
||||
override def setUp() {
|
||||
super.setUp()
|
||||
}
|
||||
|
||||
override def tearDown() {
|
||||
super.tearDown()
|
||||
}
|
||||
|
||||
def testLeaderElectionWithCreateTopic {
|
||||
var servers: Seq[KafkaServer] = Seq.empty[KafkaServer]
|
||||
// start both servers
|
||||
val server1 = TestUtils.createServer(new KafkaConfig(configProps1))
|
||||
val server2 = TestUtils.createServer(new KafkaConfig(configProps2))
|
||||
|
||||
servers ++= List(server1, server2)
|
||||
}
|
||||
|
||||
override def tearDown() {
|
||||
servers.map(server => server.shutdown())
|
||||
servers.map(server => Utils.rm(server.config.logDir))
|
||||
super.tearDown()
|
||||
}
|
||||
|
||||
def testLeaderElectionAndEpoch {
|
||||
// start 2 brokers
|
||||
val topic = "new-topic"
|
||||
val partitionId = 0
|
||||
|
@ -59,62 +58,39 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, "0:1")
|
||||
|
||||
// wait until leader is elected
|
||||
var leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertTrue("Leader should get elected", leader.isDefined)
|
||||
val leader1 = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
val leaderEpoch1 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
|
||||
debug("leader Epoc: " + leaderEpoch1)
|
||||
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
|
||||
assertTrue("Leader should get elected", leader1.isDefined)
|
||||
// NOTE: this is to avoid transient test failures
|
||||
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
||||
assertTrue("Leader could be broker 0 or broker 1", (leader1.getOrElse(-1) == 0) || (leader1.getOrElse(-1) == 1))
|
||||
assertEquals("First epoch value should be 0", 0, leaderEpoch1)
|
||||
|
||||
// kill the server hosting the preferred replica
|
||||
server1.shutdown()
|
||||
|
||||
// check if leader moves to the other server
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 1500)
|
||||
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
|
||||
|
||||
val leaderPath = zkClient.getChildren(ZkUtils.getTopicPartitionPath(topic, "0"))
|
||||
// bring the preferred replica back
|
||||
servers.head.startup()
|
||||
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertEquals("Leader must remain on broker 1", 1, leader.getOrElse(-1))
|
||||
|
||||
// shutdown current leader (broker 1)
|
||||
servers.last.shutdown()
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
// check if leader moves to the other server
|
||||
val leader2 = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 1500)
|
||||
val leaderEpoch2 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
|
||||
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
|
||||
debug("leader Epoc: " + leaderEpoch2)
|
||||
assertEquals("Leader must move to broker 0", 0, leader2.getOrElse(-1))
|
||||
if(leader1.get == leader2.get)
|
||||
assertEquals("Second epoch value should be " + leaderEpoch1, leaderEpoch1, leaderEpoch2)
|
||||
else
|
||||
assertEquals("Second epoch value should be %d".format(leaderEpoch1+1) , leaderEpoch1+1, leaderEpoch2)
|
||||
|
||||
// test if the leader is the preferred replica
|
||||
assertEquals("Leader must be preferred replica on broker 0", 0, leader.getOrElse(-1))
|
||||
// shutdown the servers and delete data hosted on them
|
||||
servers.map(server => server.shutdown())
|
||||
servers.map(server => Utils.rm(server.config.logDir))
|
||||
}
|
||||
|
||||
// Assuming leader election happens correctly, test if epoch changes as expected
|
||||
def testEpoch() {
|
||||
// keep switching leaders to see if epoch changes correctly
|
||||
val topic = "new-topic"
|
||||
val partitionId = 0
|
||||
|
||||
// setup 2 brokers in ZK
|
||||
val brokers = TestUtils.createBrokersInZk(zkClient, List(brokerId1, brokerId2))
|
||||
|
||||
// create topic with 1 partition, 2 replicas, one on each broker
|
||||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, "0:1")
|
||||
|
||||
var newLeaderEpoch = ZkUtils.tryToBecomeLeaderForPartition(zkClient, topic, partitionId, 0)
|
||||
assertTrue("Broker 0 should become leader", newLeaderEpoch.isDefined)
|
||||
assertEquals("First epoch value should be 1", 1, newLeaderEpoch.get._1)
|
||||
|
||||
ZkUtils.deletePath(zkClient, ZkUtils.getTopicPartitionLeaderPath(topic, partitionId.toString))
|
||||
newLeaderEpoch = ZkUtils.tryToBecomeLeaderForPartition(zkClient, topic, partitionId, 1)
|
||||
assertTrue("Broker 1 should become leader", newLeaderEpoch.isDefined)
|
||||
assertEquals("Second epoch value should be 2", 2, newLeaderEpoch.get._1)
|
||||
|
||||
ZkUtils.deletePath(zkClient, ZkUtils.getTopicPartitionLeaderPath(topic, partitionId.toString))
|
||||
newLeaderEpoch = ZkUtils.tryToBecomeLeaderForPartition(zkClient, topic, partitionId, 0)
|
||||
assertTrue("Broker 0 should become leader again", newLeaderEpoch.isDefined)
|
||||
assertEquals("Third epoch value should be 3", 3, newLeaderEpoch.get._1)
|
||||
|
||||
TestUtils.deleteBrokersInZk(zkClient, List(brokerId1, brokerId2))
|
||||
servers.last.startup()
|
||||
servers.head.shutdown()
|
||||
Thread.sleep(zookeeper.tickTime)
|
||||
val leader3 = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 1500)
|
||||
val leaderEpoch3 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
|
||||
debug("leader Epoc: " + leaderEpoch3)
|
||||
debug("Leader is elected to be: %s".format(leader3.getOrElse(-1)))
|
||||
assertEquals("Leader must return to 1", 1, leader3.getOrElse(-1))
|
||||
if(leader2.get == leader3.get)
|
||||
assertEquals("Second epoch value should be " + leaderEpoch2, leaderEpoch2, leaderEpoch3)
|
||||
else
|
||||
assertEquals("Second epoch value should be %d".format(leaderEpoch2+1) , leaderEpoch2+1, leaderEpoch3)
|
||||
}
|
||||
}
|
|
@ -57,7 +57,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":"))
|
||||
|
||||
// wait until leader is elected
|
||||
var leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
var leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertTrue("Leader should get elected", leader.isDefined)
|
||||
// NOTE: this is to avoid transient test failures
|
||||
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
||||
|
@ -89,7 +89,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":"))
|
||||
|
||||
// wait until leader is elected
|
||||
var leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
var leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertTrue("Leader should get elected", leader.isDefined)
|
||||
// NOTE: this is to avoid transient test failures
|
||||
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
||||
|
@ -103,13 +103,13 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
assertEquals(30L, hwFile1.read(topic, 0))
|
||||
|
||||
// check if leader moves to the other server
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
|
||||
|
||||
// bring the preferred replica back
|
||||
server1.startup()
|
||||
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertEquals("Leader must remain on broker 1", 1, leader.getOrElse(-1))
|
||||
|
||||
assertEquals(30L, hwFile1.read(topic, 0))
|
||||
|
@ -118,7 +118,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
assertEquals(30L, hwFile2.read(topic, 0))
|
||||
|
||||
server2.startup()
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertEquals("Leader must remain on broker 0", 0, leader.getOrElse(-1))
|
||||
|
||||
sendMessages()
|
||||
|
@ -159,7 +159,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":"))
|
||||
|
||||
// wait until leader is elected
|
||||
var leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
var leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertTrue("Leader should get elected", leader.isDefined)
|
||||
// NOTE: this is to avoid transient test failures
|
||||
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
||||
|
@ -202,7 +202,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(_.brokerId).mkString(":"))
|
||||
|
||||
// wait until leader is elected
|
||||
var leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
var leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertTrue("Leader should get elected", leader.isDefined)
|
||||
// NOTE: this is to avoid transient test failures
|
||||
assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1))
|
||||
|
@ -218,7 +218,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
server2.startup()
|
||||
// check if leader moves to the other server
|
||||
leader = waitUntilLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
leader = waitUntilLiveLeaderIsElected(zkClient, topic, partitionId, 500)
|
||||
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
|
||||
|
||||
assertEquals(60L, hwFile1.read(topic, 0))
|
||||
|
|
|
@ -51,7 +51,7 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
// create a topic and partition and await leadership
|
||||
for (topic <- List(topic1,topic2)) {
|
||||
CreateTopicCommand.createTopic(zkClient, topic, 1, 2, configs.map(c => c.brokerId).mkString(":"))
|
||||
TestUtils.waitUntilLeaderIsElected(zkClient, topic, 0, 1000)
|
||||
TestUtils.waitUntilLiveLeaderIsElected(zkClient, topic, 0, 1000)
|
||||
}
|
||||
|
||||
// send test messages to leader
|
||||
|
|
|
@ -73,7 +73,7 @@ class RequestPurgatoryTest {
|
|||
assertTrue("Time for expiration was about 20ms", (elapsed - expiration).abs < 10L)
|
||||
}
|
||||
|
||||
class MockRequestPurgatory extends RequestPurgatory[DelayedRequest, ProducerRequest] {
|
||||
class MockRequestPurgatory extends RequestPurgatory[DelayedRequest, ProducerRequest]("Mock Request Purgatory") {
|
||||
val satisfied = mutable.Set[DelayedRequest]()
|
||||
val expired = mutable.Set[DelayedRequest]()
|
||||
def awaitExpiration(delayed: DelayedRequest) = {
|
||||
|
|
|
@ -73,7 +73,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
val server = new KafkaServer(config)
|
||||
server.startup()
|
||||
|
||||
waitUntilLeaderIsElected(zkClient, topic, 0, 1000)
|
||||
waitUntilLiveLeaderIsElected(zkClient, topic, 0, 1000)
|
||||
|
||||
var fetchedMessage: ByteBufferMessageSet = null
|
||||
while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
|
||||
|
|
|
@ -1,124 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.server
|
||||
|
||||
import org.scalatest.junit.JUnit3Suite
|
||||
import kafka.zk.ZooKeeperTestHarness
|
||||
import kafka.common.QueueFullException
|
||||
import junit.framework.Assert._
|
||||
import kafka.utils.{ZkQueue, TestUtils}
|
||||
|
||||
class StateChangeTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||
|
||||
val brokerId1 = 0
|
||||
val port1 = TestUtils.choosePort()
|
||||
var stateChangeQ: ZkQueue = null
|
||||
val config = new KafkaConfig(TestUtils.createBrokerConfig(brokerId1, port1))
|
||||
|
||||
override def setUp() {
|
||||
super.setUp()
|
||||
|
||||
// create a queue
|
||||
val queuePath = "/brokers/state/" + config.brokerId
|
||||
stateChangeQ = new ZkQueue(zkClient, queuePath, 10)
|
||||
}
|
||||
|
||||
override def tearDown() {
|
||||
super.tearDown()
|
||||
}
|
||||
|
||||
def testZkQueueDrainAll() {
|
||||
for(i <- 0 until 5) {
|
||||
val itemPath = stateChangeQ.put("test:0:follower")
|
||||
val item = itemPath.split("/").last.split("-").last.toInt
|
||||
assertEquals(i, item)
|
||||
}
|
||||
|
||||
var numItems: Int = 0
|
||||
for(i <- 0 until 5) {
|
||||
val item = stateChangeQ.take()
|
||||
assertEquals("test:0:follower", item._2)
|
||||
assertTrue(stateChangeQ.remove(item))
|
||||
numItems += 1
|
||||
}
|
||||
assertEquals(5, numItems)
|
||||
|
||||
for(i <- 5 until 10) {
|
||||
val itemPath = stateChangeQ.put("test:1:follower")
|
||||
val item = itemPath.split("/").last.split("-").last.toInt
|
||||
assertEquals(i+5, item)
|
||||
}
|
||||
|
||||
numItems = 0
|
||||
for(i <- 0 until 5) {
|
||||
val item = stateChangeQ.take()
|
||||
assertTrue(stateChangeQ.remove(item))
|
||||
assertEquals("test:1:follower", item._2)
|
||||
numItems += 1
|
||||
}
|
||||
assertEquals(5, numItems)
|
||||
}
|
||||
|
||||
def testZkQueueFull() {
|
||||
for(i <- 0 until 10) {
|
||||
val itemPath = stateChangeQ.put("test:0:follower")
|
||||
val item = itemPath.split("/").last.split("-").last.toInt
|
||||
assertEquals(i, item)
|
||||
}
|
||||
|
||||
try {
|
||||
stateChangeQ.put("test:0:follower")
|
||||
fail("Queue should be full")
|
||||
}catch {
|
||||
case e:QueueFullException => // expected
|
||||
}
|
||||
}
|
||||
|
||||
def testStateChangeCommandJson() {
|
||||
// test start replica
|
||||
val topic = "foo"
|
||||
val partition = 0
|
||||
val epoch = 1
|
||||
|
||||
val startReplica = new StartReplica(topic, partition, epoch)
|
||||
val startReplicaJson = startReplica.toJson()
|
||||
val startReplicaFromJson = StateChangeCommand.getStateChangeRequest(startReplicaJson)
|
||||
assertEquals(startReplica, startReplicaFromJson)
|
||||
|
||||
// test close replica
|
||||
val closeReplica = new StartReplica(topic, partition, epoch)
|
||||
val closeReplicaJson = startReplica.toJson()
|
||||
val closeReplicaFromJson = StateChangeCommand.getStateChangeRequest(closeReplicaJson)
|
||||
assertEquals(closeReplica, closeReplicaFromJson)
|
||||
}
|
||||
|
||||
// TODO: Do this after patch for delete topic/delete partition is in
|
||||
def testStateChangeRequestValidity() {
|
||||
// mock out the StateChangeRequestHandler
|
||||
|
||||
// setup 3 replicas for one topic partition
|
||||
|
||||
// shutdown follower 1
|
||||
|
||||
// restart leader to trigger epoch change
|
||||
|
||||
// start follower 1
|
||||
|
||||
// test follower 1 acted only on one become follower request
|
||||
}
|
||||
}
|
|
@ -5,7 +5,7 @@
|
|||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
|
@ -43,33 +43,33 @@ import collection.mutable.{Map, Set}
|
|||
* Utility functions to help with testing
|
||||
*/
|
||||
object TestUtils extends Logging {
|
||||
|
||||
|
||||
val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"
|
||||
val Digits = "0123456789"
|
||||
val LettersAndDigits = Letters + Digits
|
||||
|
||||
|
||||
/* A consistent random number generator to make tests repeatable */
|
||||
val seededRandom = new Random(192348092834L)
|
||||
val random = new Random()
|
||||
|
||||
|
||||
/**
|
||||
* Choose a number of random available ports
|
||||
*/
|
||||
def choosePorts(count: Int): List[Int] = {
|
||||
val sockets =
|
||||
val sockets =
|
||||
for(i <- 0 until count)
|
||||
yield new ServerSocket(0)
|
||||
yield new ServerSocket(0)
|
||||
val socketList = sockets.toList
|
||||
val ports = socketList.map(_.getLocalPort)
|
||||
socketList.map(_.close)
|
||||
ports
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Choose an available port
|
||||
*/
|
||||
def choosePort(): Int = choosePorts(1).head
|
||||
|
||||
|
||||
/**
|
||||
* Create a temporary directory
|
||||
*/
|
||||
|
@ -80,7 +80,7 @@ object TestUtils extends Logging {
|
|||
f.deleteOnExit()
|
||||
f
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a temporary file
|
||||
*/
|
||||
|
@ -89,12 +89,12 @@ object TestUtils extends Logging {
|
|||
f.deleteOnExit()
|
||||
f
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a temporary file and return an open file channel for this file
|
||||
*/
|
||||
def tempChannel(): FileChannel = new RandomAccessFile(tempFile(), "rw").getChannel()
|
||||
|
||||
|
||||
/**
|
||||
* Create a kafka server instance with appropriate test settings
|
||||
* USING THIS IS A SIGN YOU ARE NOT WRITING A REAL UNIT TEST
|
||||
|
@ -105,15 +105,15 @@ object TestUtils extends Logging {
|
|||
server.startup()
|
||||
server
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a test config for the given node id
|
||||
*/
|
||||
def createBrokerConfigs(numConfigs: Int): List[Properties] = {
|
||||
for((port, node) <- choosePorts(numConfigs).zipWithIndex)
|
||||
yield createBrokerConfig(node, port)
|
||||
yield createBrokerConfig(node, port)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a test config for the given node id
|
||||
*/
|
||||
|
@ -127,7 +127,7 @@ object TestUtils extends Logging {
|
|||
props.put("replica.socket.timeout.ms", "1500")
|
||||
props
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a test config for a consumer
|
||||
*/
|
||||
|
@ -150,9 +150,9 @@ object TestUtils extends Logging {
|
|||
* Wrap the message in a message set
|
||||
* @param payload The bytes of the message
|
||||
*/
|
||||
def singleMessageSet(payload: Array[Byte]) =
|
||||
def singleMessageSet(payload: Array[Byte]) =
|
||||
new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(payload))
|
||||
|
||||
|
||||
/**
|
||||
* Generate an array of random bytes
|
||||
* @param numBytes The size of the array
|
||||
|
@ -162,7 +162,7 @@ object TestUtils extends Logging {
|
|||
seededRandom.nextBytes(bytes)
|
||||
bytes
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Generate a random string of letters and digits of the given length
|
||||
* @param len The length of the string
|
||||
|
@ -183,7 +183,7 @@ object TestUtils extends Logging {
|
|||
for(i <- 0 until b1.limit - b1.position)
|
||||
assertEquals("byte " + i + " byte not equal.", b1.get(b1.position + i), b2.get(b1.position + i))
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Throw an exception if the two iterators are of differing lengths or contain
|
||||
* different messages on their Nth element
|
||||
|
@ -197,28 +197,28 @@ object TestUtils extends Logging {
|
|||
|
||||
// check if the expected iterator is longer
|
||||
if (expected.hasNext) {
|
||||
var length1 = length;
|
||||
while (expected.hasNext) {
|
||||
expected.next
|
||||
length1 += 1
|
||||
}
|
||||
assertFalse("Iterators have uneven length-- first has more: "+length1 + " > " + length, true);
|
||||
var length1 = length;
|
||||
while (expected.hasNext) {
|
||||
expected.next
|
||||
length1 += 1
|
||||
}
|
||||
assertFalse("Iterators have uneven length-- first has more: "+length1 + " > " + length, true);
|
||||
}
|
||||
|
||||
// check if the actual iterator was longer
|
||||
if (actual.hasNext) {
|
||||
var length2 = length;
|
||||
while (actual.hasNext) {
|
||||
actual.next
|
||||
length2 += 1
|
||||
}
|
||||
assertFalse("Iterators have uneven length-- second has more: "+length2 + " > " + length, true);
|
||||
var length2 = length;
|
||||
while (actual.hasNext) {
|
||||
actual.next
|
||||
length2 += 1
|
||||
}
|
||||
assertFalse("Iterators have uneven length-- second has more: "+length2 + " > " + length, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Throw an exception if an iterable has different length than expected
|
||||
*
|
||||
*
|
||||
*/
|
||||
def checkLength[T](s1: Iterator[T], expectedLength:Int) {
|
||||
var n = 0
|
||||
|
@ -269,7 +269,7 @@ object TestUtils extends Logging {
|
|||
* Create a hexidecimal string for the given bytes
|
||||
*/
|
||||
def hexString(bytes: Array[Byte]): String = hexString(ByteBuffer.wrap(bytes))
|
||||
|
||||
|
||||
/**
|
||||
* Create a hexidecimal string for the given bytes
|
||||
*/
|
||||
|
@ -279,7 +279,7 @@ object TestUtils extends Logging {
|
|||
builder.append(String.format("%x", Integer.valueOf(buffer.get(buffer.position + i))))
|
||||
builder.toString
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a producer for the given host and port
|
||||
*/
|
||||
|
@ -340,7 +340,7 @@ object TestUtils extends Logging {
|
|||
buffer += ("msg" + i)
|
||||
buffer
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a wired format request based on simple basic information
|
||||
*/
|
||||
|
@ -381,34 +381,55 @@ object TestUtils extends Logging {
|
|||
}
|
||||
|
||||
def makeLeaderForPartition(zkClient: ZkClient, topic: String, leaderPerPartitionMap: scala.collection.immutable.Map[Int, Int]) {
|
||||
leaderPerPartitionMap.foreach(leaderForPartition => ZkUtils.tryToBecomeLeaderForPartition(zkClient, topic,
|
||||
leaderForPartition._1, leaderForPartition._2))
|
||||
leaderPerPartitionMap.foreach
|
||||
{
|
||||
leaderForPartition => {
|
||||
val partition = leaderForPartition._1
|
||||
val leader = leaderForPartition._2
|
||||
try{
|
||||
val currentLeaderAndISROpt = ZkUtils.getLeaderAndISRForPartition(zkClient, topic, partition)
|
||||
var newLeaderAndISR: LeaderAndISR = null
|
||||
if(currentLeaderAndISROpt == None)
|
||||
newLeaderAndISR = new LeaderAndISR(leader, List(leader))
|
||||
else{
|
||||
newLeaderAndISR = currentLeaderAndISROpt.get
|
||||
newLeaderAndISR.leader = leader
|
||||
newLeaderAndISR.leaderEpoch += 1
|
||||
newLeaderAndISR.zkVersion += 1
|
||||
}
|
||||
ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndISRPath( topic, partition), newLeaderAndISR.toString)
|
||||
} catch {
|
||||
case oe => error("Error while electing leader for topic %s partition %d".format(topic, partition), oe)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def waitUntilLeaderIsElected(zkClient: ZkClient, topic: String, partition: Int, timeoutMs: Long): Option[Int] = {
|
||||
val leaderLock = new ReentrantLock()
|
||||
val leaderExists = leaderLock.newCondition()
|
||||
def waitUntilLiveLeaderIsElected(zkClient: ZkClient, topic: String, partition: Int, timeoutMs: Long): Option[Int] = {
|
||||
// If the current leader is alive, just return it
|
||||
val curLeaderOpt = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
|
||||
if(curLeaderOpt.isDefined && ZkUtils.pathExists(zkClient, ZkUtils.getBrokerPath(curLeaderOpt.get)))
|
||||
return curLeaderOpt
|
||||
|
||||
val leaderLock = new ReentrantLock()
|
||||
val liveLeaderIsElected = leaderLock.newCondition()
|
||||
|
||||
info("Waiting for leader to be elected for topic %s partition %d".format(topic, partition))
|
||||
leaderLock.lock()
|
||||
try {
|
||||
// check if leader already exists
|
||||
zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndISRPath(topic, partition), new LeaderElectionListener(topic, partition, leaderLock, liveLeaderIsElected, zkClient))
|
||||
liveLeaderIsElected.await(timeoutMs, TimeUnit.MILLISECONDS)
|
||||
val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
|
||||
leader match {
|
||||
case Some(l) => info("Leader %d exists for topic %s partition %d".format(l, topic, partition))
|
||||
leader
|
||||
case None => zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderPath(topic, partition.toString),
|
||||
new LeaderExistsListener(topic, partition, leaderLock, leaderExists))
|
||||
info("No leader exists. Waiting for %d ms".format(timeoutMs))
|
||||
leaderExists.await(timeoutMs, TimeUnit.MILLISECONDS)
|
||||
// check if leader is elected
|
||||
val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
|
||||
leader match {
|
||||
case Some(l) => info("Leader %d elected for topic %s partition %d".format(l, topic, partition))
|
||||
case None => error("Timing out after %d ms since leader is not elected for topic %s partition %d"
|
||||
.format(timeoutMs, topic, partition))
|
||||
}
|
||||
leader
|
||||
case Some(l) =>
|
||||
if(ZkUtils.pathExists(zkClient, ZkUtils.getBrokerPath(l))){
|
||||
info("Leader %d is elected for topic %s partition %d".format(l, topic, partition))
|
||||
return leader
|
||||
} else {
|
||||
warn("Timing out after %d ms but current leader in zookeeper is not alive, and no live leader for partition [%s, %d] is elected".format(topic, partition))
|
||||
return None
|
||||
}
|
||||
case None => warn("Timing out after %d ms but no leader is elected for topic %s partition %d".format(timeoutMs, topic, partition))
|
||||
return None
|
||||
}
|
||||
} finally {
|
||||
leaderLock.unlock()
|
||||
|
@ -430,50 +451,49 @@ object TestUtils extends Logging {
|
|||
}
|
||||
|
||||
object ControllerTestUtils{
|
||||
def createSampleLeaderAndISRRequest() : LeaderAndISRRequest = {
|
||||
def createTestLeaderAndISRRequest() : LeaderAndISRRequest = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
|
||||
val leader1 = 1;
|
||||
val ISR1 = List(1, 2, 3)
|
||||
val leader1 = 0;
|
||||
val isr1 = List(0, 1, 2)
|
||||
|
||||
val leader2 = 2;
|
||||
val ISR2 = List(2, 3, 4)
|
||||
val leader2 = 0;
|
||||
val isr2 = List(0, 2, 3)
|
||||
|
||||
val leaderAndISR1 = new LeaderAndISR(leader1, 1, ISR1, 1)
|
||||
val leaderAndISR2 = new LeaderAndISR(leader2, 1, ISR2, 2)
|
||||
val map = Map(((topic1, 1), leaderAndISR1), ((topic1, 2), leaderAndISR1),
|
||||
((topic2, 1), leaderAndISR2), ((topic2, 2), leaderAndISR2))
|
||||
new LeaderAndISRRequest(1, "client 1", 1, 4, map)
|
||||
val leaderAndISR1 = new LeaderAndISR(leader1, 1, isr1, 1)
|
||||
val leaderAndISR2 = new LeaderAndISR(leader2, 1, isr2, 2)
|
||||
val map = Map(((topic1, 0), leaderAndISR1),
|
||||
((topic2, 0), leaderAndISR2))
|
||||
new LeaderAndISRRequest( LeaderAndISRRequest.NotInit, map)
|
||||
}
|
||||
|
||||
def createSampleLeaderAndISRResponse() : LeaderAndISRResponse = {
|
||||
def createTestLeaderAndISRResponse() : LeaderAndISRResponse = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
val responseMap = Map(((topic1, 1), ErrorMapping.NoError), ((topic1, 2), ErrorMapping.NoError),
|
||||
((topic2, 1), ErrorMapping.NoError), ((topic2, 2), ErrorMapping.NoError))
|
||||
val responseMap = Map(((topic1, 0), ErrorMapping.NoError),
|
||||
((topic2, 0), ErrorMapping.NoError))
|
||||
new LeaderAndISRResponse(1, responseMap)
|
||||
}
|
||||
|
||||
|
||||
def createSampleStopReplicaRequest() : StopReplicaRequest = {
|
||||
def createTestStopReplicaRequest() : StopReplicaRequest = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
new StopReplicaRequest(1, "client 1", 1000, Set((topic1, 1), (topic1, 2),
|
||||
(topic2, 1), (topic2, 2)))
|
||||
new StopReplicaRequest(Set((topic1, 0), (topic2, 0)))
|
||||
}
|
||||
|
||||
def createSampleStopReplicaResponse() : StopReplicaResponse = {
|
||||
def createTestStopReplicaResponse() : StopReplicaResponse = {
|
||||
val topic1 = "test1"
|
||||
val topic2 = "test2"
|
||||
val responseMap = Map(((topic1, 1), ErrorMapping.NoError), ((topic1, 2), ErrorMapping.NoError),
|
||||
((topic2, 1), ErrorMapping.NoError), ((topic2, 2), ErrorMapping.NoError))
|
||||
val responseMap = Map(((topic1, 0), ErrorMapping.NoError),
|
||||
((topic2, 0), ErrorMapping.NoError))
|
||||
new StopReplicaResponse(1, responseMap)
|
||||
}
|
||||
}
|
||||
|
||||
object TestZKUtils {
|
||||
val zookeeperConnect = "127.0.0.1:2182"
|
||||
val zookeeperConnect = "127.0.0.1:2182"
|
||||
}
|
||||
|
||||
class StringSerializer extends Encoder[String] {
|
||||
|
|
|
@ -40,7 +40,7 @@ class ZKEphemeralTest extends JUnit3Suite with ZooKeeperTestHarness {
|
|||
|
||||
var testData: String = null
|
||||
|
||||
testData = ZkUtils.readData(zkClient, "/tmp/zktest")
|
||||
testData = ZkUtils.readData(zkClient, "/tmp/zktest")._1
|
||||
Assert.assertNotNull(testData)
|
||||
|
||||
zkClient.close
|
||||
|
|
Loading…
Reference in New Issue