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:
Jun Rao 2012-07-31 15:33:29 +00:00
parent 19fbdb3099
commit 2a9d42c243
54 changed files with 1347 additions and 1742 deletions

View File

@ -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)

View File

@ -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){

View File

@ -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
}
}
}

View File

@ -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)
}

View File

@ -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()
}
}
}

View File

@ -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)
}

View File

@ -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 */

View File

@ -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) {

View File

@ -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)

View File

@ -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)
}
}

View File

@ -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 = {

View File

@ -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
}

View File

@ -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")
}
}

View File

@ -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)

View File

@ -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) =

View File

@ -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()
}
}
}
}

View File

@ -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")
}
}

View File

@ -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

View File

@ -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
}
}

View File

@ -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() {

View File

@ -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")
}
}

View File

@ -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
}
}
}

View File

@ -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
}

View File

@ -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
}
}

View File

@ -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))

View File

@ -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)
}

View File

@ -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

View File

@ -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))

View File

@ -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
}
}
}

View File

@ -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 {

View File

@ -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)

View File

@ -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)

View File

@ -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)
}
}

View File

@ -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

View File

@ -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
}
}
}

View File

@ -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: _*)

View File

@ -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)
}
}
}

View File

@ -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])

View File

@ -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))

View File

@ -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) {

View File

@ -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()

View File

@ -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)): _*)

View File

@ -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 {

View File

@ -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)

View File

@ -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)

View File

@ -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

View File

@ -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)
}
}

View File

@ -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))

View File

@ -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

View File

@ -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) = {

View File

@ -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) {

View File

@ -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
}
}

View File

@ -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] {

View File

@ -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