kafka apis反映出kafka broker server可以提供哪些服务,
broker server主要和producer,consumer,controller有交互,搞清这些api就清楚了broker server的所有行为
handleOffsetRequest
提供对offset的查询的需求,比如查询earliest,latest offset是什么,或before某个时间戳的offset是什么
try {
// ensure leader exists
// 确定是否是leader replica,因为只有leader可以响应offset请求
// 如果不是会抛异常
val localReplica = if(!offsetRequest.isFromDebuggingClient)
replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition)
else
replicaManager.getReplicaOrException(topicAndPartition.topic, topicAndPartition.partition)
val offsets = {
val allOffsets = fetchOffsets(replicaManager.logManager, //获取offsets列表
topicAndPartition,
partitionOffsetRequestInfo.time,
partitionOffsetRequestInfo.maxNumOffsets)
if (!offsetRequest.isFromOrdinaryClient) {
allOffsets
} else {
val hw = localReplica.highWatermark.messageOffset
if (allOffsets.exists(_ > hw)) //过滤掉hw以后的offsets,因为那些都不是应该用户可见的
hw +: allOffsets.dropWhile(_ > hw)
else
allOffsets
}
}
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.NoError, offsets))
} catch {
// NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages
// are typically transient and there is no value in logging the entire stack trace for the same
case utpe: UnknownTopicOrPartitionException =>
warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition, utpe.getMessage))
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) )
case nle: NotLeaderForPartitionException =>
warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage))
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) )
case e: Throwable =>
warn("Error while responding to offset request", e)
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) )
}
可以看到,当没有找到topic->partition, 或partition leader,或其他异常的时候,就会导致返回offsets为nil
这样在客户端,经常通过获取latestOffset来算spoutLag,会出现负值的情况
然后,fetchOffset调用fetchOffsetsBefore,来完成offset的获取,
def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = {
val segsArray = log.logSegments.toArray //取出所有segments
var offsetTimeArray: Array[(Long, Long)] = null
if(segsArray.last.size > 0) //看最新的segment,即真正被写入的,是否有数据(Segment.size取出segment中log的bytes)
offsetTimeArray = new Array[(Long, Long)](segsArray.length + 1)
else
offsetTimeArray = new Array[(Long, Long)](segsArray.length) for(i <- 0 until segsArray.length)
offsetTimeArray(i) = (segsArray(i).baseOffset, segsArray(i).lastModified) //对每个segment, 生成(baseOffset,最后更新的时间)
if(segsArray.last.size > 0)
offsetTimeArray(segsArray.length) = (log.logEndOffset, SystemTime.milliseconds) //对于最新的segment逻辑不同,这里取的是log.logEndOffset,有点tricky,因为只有取latest offset时才会取到最后这个 var startIndex = -1
timestamp match {
case OffsetRequest.LatestTime =>
startIndex = offsetTimeArray.length - 1 //Latest,取的其实是log.logEndOffset
case OffsetRequest.EarliestTime =>
startIndex = 0 //earlist, 取的是第一个segment的baseOffset
case _ => //对某一个时间,去offset
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) { //从最后一个segment开始,向前遍历
if (offsetTimeArray(startIndex)._2 <= timestamp) //找到小于等于timestamp的segment
isFound = true
else
startIndex -=1
}
} val retSize = maxNumOffsets.min(startIndex + 1) //选择返回几个offset
val ret = new Array[Long](retSize)
for(j <- 0 until retSize) {
ret(j) = offsetTimeArray(startIndex)._1 //返回当前segment,往前的所有segment的baseoffset
startIndex -= 1
}
// ensure that the returned seq is in descending order of offsets
ret.toSeq.sortBy(- _)
}
handleProducerOrOffsetCommitRequest
这个用于处理Producer的请求,其实就是写数据
名字有些tricky,和offsetCommit有什么关系,因为对于kafka的highlevel consumer,consumeroffset是被写入kafka topic的,所以offsetCommitRequest其实就是一种特殊的producer request
你看他实际也是,用producerRequestFromOffsetCommit,将它转换成了producer request
主要调用appendToLocalLog,核心逻辑
val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) //取到partition,如果没有找到,抛异常
val info = partitionOpt match {
case Some(partition) =>
partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet],producerRequest.requiredAcks) //将数据写入
case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
.format(topicAndPartition, brokerId))
}
Partition.appendMessagesToLeader
def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int=0) = {
inReadLock(leaderIsrUpdateLock) {
val leaderReplicaOpt = leaderReplicaIfLocal() //是否是leader replica
leaderReplicaOpt match {
case Some(leaderReplica) =>
val log = leaderReplica.log.get //取得replica.log
val minIsr = log.config.minInSyncReplicas //配置的最小isr的size
val inSyncSize = inSyncReplicas.size //当前isr真实的size // Avoid writing to leader if there are not enough insync replicas to make it safe
if (inSyncSize < minIsr && requiredAcks == -1) {
throw new NotEnoughReplicasException("Number of insync replicas for partition [%s,%d] is [%d], below required minimum [%d]"
.format(topic,partitionId,minIsr,inSyncSize))
} val info = log.append(messages, assignOffsets = true) //将message append到log
// 当有新数据产生了,需要去触发delayedFetchRequest,consumer的fetch request当达到log end offset的时候是会block的,所以这里需要unblock
// probably unblock some follower fetch requests since log end offset has been updated
replicaManager.unblockDelayedFetchRequests(new TopicAndPartition(this.topic, this.partitionId))
// we may need to increment high watermark since ISR could be down to 1
maybeIncrementLeaderHW(leaderReplica) //增加HW
info
case None => //如果找不到leader,往往是因为发生了迁移
throw new NotLeaderForPartitionException("Leader not local for partition [%s,%d] on broker %d"
.format(topic, partitionId, localBrokerId))
}
}
}
对于producer的写策略,取决于配置的acker机制,
acks = 0,那没有failover处理的,发就发了
acks = 1,当写leader replica成功后就返回,其他的replica都是通过fetcher去同步的,所以kafka是异步写
不过有数据丢失的风险,如果leader的数据没有来得及同步,leader挂了,那么会丢失数据
acks = –1, 要等待所有的replicas都成功后,才能返回
所以这里需要产生DelayedProducerRequest,这个request只有在所有的follower都fetch成功后才能reponse
所以DelayedProducerRequest会在fetch request中被触发unblock
if(produceRequest.requiredAcks == 0) {
//acks == 0,即不需要ack,没啥需要特别做的
} else if (produceRequest.requiredAcks == 1 || //acks == 1,即需要立即返回response
produceRequest.numPartitions <= 0 || //没有要求取数据,因为request里面的partition数为0
numPartitionsInError == produceRequest.numPartitions) { //所有的partition都取失败了
//这几种情况都需要立即返回
requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
} else { //这个地方没加注释,应该是ack == -1的情况
// create a list of (topic, partition) pairs to use as keys for this delayed request
val producerRequestKeys = produceRequest.data.keys.toSeq
val statuses = localProduceResults.map(r =>
r.key -> DelayedProduceResponseStatus(r.end + 1, ProducerResponseStatus(r.errorCode, r.start))).toMap
val delayedRequest = new DelayedProduce(
producerRequestKeys,
request,
produceRequest.ackTimeoutMs.toLong,
produceRequest,
statuses,
offsetCommitRequestOpt) // add the produce request for watch if it's not satisfied, otherwise send the response back
val satisfiedByMe = producerRequestPurgatory.checkAndMaybeWatch(delayedRequest)
if (satisfiedByMe)
producerRequestPurgatory.respond(delayedRequest)
}
handleFetchRequest
响应读数据的请求,来自consumer或follower fetcher
def handleFetchRequest(request: RequestChannel.Request) {
val fetchRequest = request.requestObj.asInstanceOf[FetchRequest]
val dataRead = replicaManager.readMessageSets(fetchRequest) //从replicaManager读出数据 // if the fetch request comes from the follower,
// update its corresponding log end offset
if(fetchRequest.isFromFollower) //如果是follower的fetch request,更新follower的leo,还可能需要更新ISR
recordFollowerLogEndOffsets(fetchRequest.replicaId, dataRead.mapValues(_.offset)) // check if this fetch request can be satisfied right away
val bytesReadable = dataRead.values.map(_.data.messages.sizeInBytes).sum
val errorReadingData = dataRead.values.foldLeft(false)((errorIncurred, dataAndOffset) =>
errorIncurred || (dataAndOffset.data.error != ErrorMapping.NoError))
//fetch request是可以delay的,但满足如下要求时是需要立刻返回
// send the data immediately if 1) fetch request does not want to wait
// 2) fetch request does not require any data
// 3) has enough data to respond
// 4) some error happens while reading data
if(fetchRequest.maxWait <= 0 || //不想等
fetchRequest.numPartitions <= 0 || //没有请求数据
bytesReadable >= fetchRequest.minBytes || //读到的数据已足够
errorReadingData) { //有异常
debug("Returning fetch response %s for fetch request with correlation id %d to client %s"
.format(dataRead.values.map(_.data.error).mkString(","), fetchRequest.correlationId, fetchRequest.clientId))
val response = new FetchResponse(fetchRequest.correlationId, dataRead.mapValues(_.data))
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response)))
} else { //否则产生delay fetcher request,比如没新数据的时候,后续有数据时会unblock这些request
debug("Putting fetch request with correlation id %d from client %s into purgatory".format(fetchRequest.correlationId,
fetchRequest.clientId))
// create a list of (topic, partition) pairs to use as keys for this delayed request
val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq
val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest.maxWait, fetchRequest,
dataRead.mapValues(_.offset)) // add the fetch request for watch if it's not satisfied, otherwise send the response back
val satisfiedByMe = fetchRequestPurgatory.checkAndMaybeWatch(delayedFetch)
if (satisfiedByMe)
fetchRequestPurgatory.respond(delayedFetch)
}
}
readMessageSets其实就是对每个topicAndPartititon调用readMessageSet
private def readMessageSet(topic: String,
partition: Int,
offset: Long,
maxSize: Int,
fromReplicaId: Int): (FetchDataInfo, Long) = {
// check if the current broker is the leader for the partitions
val localReplica = if(fromReplicaId == Request.DebuggingConsumerId)
getReplicaOrException(topic, partition)
else
getLeaderReplicaIfLocal(topic, partition) //判断是否是leader,非leader也不能响应fetch请求
trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize))
//我的理解,fromReplicaId只有从follower来的fetch请求才会有
val maxOffsetOpt =
if (Request.isValidBrokerId(fromReplicaId))
None //从follower来的fetch请求,不需要设最大的offset,有多少读多少好了
else //对于普通的fetch请求,不能读超出hw offset
Some(localReplica.highWatermark.messageOffset)
val fetchInfo = localReplica.log match {
case Some(log) =>
log.read(offset, maxSize, maxOffsetOpt)
case None =>
error("Leader for partition [%s,%d] does not have a local log".format(topic, partition))
FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty)
}
(fetchInfo, localReplica.highWatermark.messageOffset)
}
如果是follower fetch request,需要做recordFollowerLogEndOffsets更新follower的leo,
private def recordFollowerLogEndOffsets(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) {
debug("Record follower log end offsets: %s ".format(offsets))
offsets.foreach {
case (topicAndPartition, offset) =>
replicaManager.updateReplicaLEOAndPartitionHW(topicAndPartition.topic, //更新LEO和HW
topicAndPartition.partition, replicaId, offset)
//当一次follower fetch成功后,需要check之前的delayedProduceRequest是否可以response
//因为ack=-1时,需要所有的follower都fetch成功后才能response
// for producer requests with ack = -1, we need to check
// if they can be unblocked after some follower's log end offsets have moved
replicaManager.unblockDelayedProduceRequests(topicAndPartition)
}
}
最终调用到ReplicaManager.updateReplicaLEOAndPartitionHW,并修正改partition的ISR
def updateReplicaLEOAndPartitionHW(topic: String, partitionId: Int, replicaId: Int, offset: LogOffsetMetadata) = {
getPartition(topic, partitionId) match {
case Some(partition) =>
partition.getReplica(replicaId) match {
case Some(replica) =>
replica.logEndOffset = offset //将follower的replica的leo设为当前取得的offset
// check if we need to update HW and expand Isr
partition.updateLeaderHWAndMaybeExpandIsr(replicaId) //更新ISR
debug("Recorded follower %d position %d for partition [%s,%d].".format(replicaId, offset.messageOffset, topic, partitionId))
case None =>
throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" +
" is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId,
offset.messageOffset, partition.assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) }
case None =>
warn("While recording the follower position, the partition [%s,%d] hasn't been created, skip updating leader HW".format(topic, partitionId))
}
}
最终调到partition.updateLeaderHWAndMaybeExpandIsr来更新ISR
def updateLeaderHWAndMaybeExpandIsr(replicaId: Int) {
inWriteLock(leaderIsrUpdateLock) {
// check if this replica needs to be added to the ISR
leaderReplicaIfLocal() match { //只有当前的replica是leader,才能更新ISR
case Some(leaderReplica) =>
val replica = getReplica(replicaId).get
val leaderHW = leaderReplica.highWatermark
// For a replica to get added back to ISR, it has to satisfy 3 conditions- //满足下面3条就需要加到ISR中
// 1. It is not already in the ISR
// 2. It is part of the assigned replica list. See KAFKA-1097
// 3. It's log end offset >= leader's high watermark
if (!inSyncReplicas.contains(replica) && //本身不在ISR中
assignedReplicas.map(_.brokerId).contains(replicaId) && //在AR中
replica.logEndOffset.offsetDiff(leaderHW) >= 0) { //当前的leo大于leader的HW, 说明已经追上了
// expand ISR
val newInSyncReplicas = inSyncReplicas + replica //扩展ISR
info("Expanding ISR for partition [%s,%d] from %s to %s"
.format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(",")))
// update ISR in ZK and cache
updateIsr(newInSyncReplicas) //把ISR更新到zk
replicaManager.isrExpandRate.mark()
}
maybeIncrementLeaderHW(leaderReplica) 增加hw
case None => // nothing to do if no longer leader
}
}
}
maybeIncrementLeaderHW
private def maybeIncrementLeaderHW(leaderReplica: Replica) {
val allLogEndOffsets = inSyncReplicas.map(_.logEndOffset) //取出ISR中所有replica的leo列表
val newHighWatermark = allLogEndOffsets.min(new LogOffsetMetadata.OffsetOrdering) //取最小的作为新的hw,这样可以保证只有在所有replica都完成同步的offset,才会设为hw
val oldHighWatermark = leaderReplica.highWatermark //当前旧的hw
if(oldHighWatermark.precedes(newHighWatermark)) { //判断新的hw一定要大于就的hw
leaderReplica.highWatermark = newHighWatermark //更新hw
debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark))
// some delayed requests may be unblocked after HW changed
val requestKey = new TopicAndPartition(this.topic, this.partitionId)
replicaManager.unblockDelayedFetchRequests(requestKey) //hw变化,触发unblockDelayedFetch很容易理解,有新数据,你之前block的读请求,可以继续读数据
replicaManager.unblockDelayedProduceRequests(requestKey) //也触发unblock DelayedProduce,hw变化表示有数据完成所有replica同步,这样可以reponse produce request
} else {
debug("Skipping update high watermark since Old hw %s is larger than new hw %s for partition [%s,%d]. All leo's are %s"
.format(oldHighWatermark, newHighWatermark, topic, partitionId, allLogEndOffsets.mkString(",")))
}
}
handleControlledShutdownRequest
响应broker发来的shutdown请求,
def handleControlledShutdownRequest(request: RequestChannel.Request) {
val controlledShutdownRequest = request.requestObj.asInstanceOf[ControlledShutdownRequest]
val partitionsRemaining = controller.shutdownBroker(controlledShutdownRequest.brokerId)
val controlledShutdownResponse = new ControlledShutdownResponse(controlledShutdownRequest.correlationId,
ErrorMapping.NoError, partitionsRemaining)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse)))
}
单纯的调用,controller.shutdownBroker,这种是优雅的shutdown,会做很多的准备工作
def shutdownBroker(id: Int) : Set[TopicAndPartition] = { if (!isActive()) { //如果当前broker不是controller,抛异常退出
throw new ControllerMovedException("Controller moved to another broker. Aborting controlled shutdown")
} controllerContext.brokerShutdownLock synchronized {
info("Shutting down broker " + id) inLock(controllerContext.controllerLock) {
if (!controllerContext.liveOrShuttingDownBrokerIds.contains(id)) //如果broker不存在,抛异常
throw new BrokerNotAvailableException("Broker id %d does not exist.".format(id)) controllerContext.shuttingDownBrokerIds.add(id) //将broker加入真正shuttingDown的broker list
debug("All shutting down brokers: " + controllerContext.shuttingDownBrokerIds.mkString(","))
debug("Live brokers: " + controllerContext.liveBrokerIds.mkString(","))
} val allPartitionsAndReplicationFactorOnBroker: Set[(TopicAndPartition, Int)] = //找出broker上所有的partition和replica
inLock(controllerContext.controllerLock) {
controllerContext.partitionsOnBroker(id)
.map(topicAndPartition => (topicAndPartition, controllerContext.partitionReplicaAssignment(topicAndPartition).size))
} allPartitionsAndReplicationFactorOnBroker.foreach {
case(topicAndPartition, replicationFactor) =>
// Move leadership serially to relinquish lock.
inLock(controllerContext.controllerLock) {
controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch =>
if (replicationFactor > 1) { //如果打开副本机制,=1就是没有副本
if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { //如果是leader
// If the broker leads the topic partition, transition the leader and update isr. Updates zk and
// notifies all affected brokers
partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition,
controlledShutdownPartitionLeaderSelector) //主动做leader重新选举
} else { //如果该broker上的replica不是leader,发送stopReplicas请求
// Stop the replica first. The state change below initiates ZK changes which should take some time
// before which the stop replica request should be completed (in most cases)
brokerRequestBatch.newBatch()
brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic,
topicAndPartition.partition, deletePartition = false)
brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) // If the broker is a follower, updates the isr in ZK and notifies the current leader
replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic,
topicAndPartition.partition, id)), OfflineReplica)
}
}
}
}
}
def replicatedPartitionsBrokerLeads() = inLock(controllerContext.controllerLock) {
trace("All leaders = " + controllerContext.partitionLeadershipInfo.mkString(","))
controllerContext.partitionLeadershipInfo.filter {
case (topicAndPartition, leaderIsrAndControllerEpoch) =>
leaderIsrAndControllerEpoch.leaderAndIsr.leader == id && controllerContext.partitionReplicaAssignment(topicAndPartition).size > 1
}.map(_._1)
}
replicatedPartitionsBrokerLeads().toSet
}
}
这里做leader重新选举用的是controlledShutdownPartitionLeaderSelector
这个选举策略很简单,
排除了shuttingDownBroker的产生新的ISR,然后选择head作为新的leader
val newIsr = currentLeaderAndIsr.isr.filter(brokerId => !controllerContext.shuttingDownBrokerIds.contains(brokerId))
val newLeaderOpt = newIsr.headOption
handleTopicMetadataRequest,handleUpdateMetadataRequest
就是处理读取和更新MetadataCache的请求,
KafkaApis.metadataCache
首先看看MetaCache是什么?
/**
* A cache for the state (e.g., current leader) of each partition. This cache is updated through
* UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously.
*/
private[server] class MetadataCache {
private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] =
new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]()
private var aliveBrokers: Map[Int, Broker] = Map()
private val partitionMetadataLock = new ReentrantReadWriteLock()
可见cache为,Map[String, mutable.Map[Int, PartitionStateInfo],记录每个topic,每个partition的PartitionStateInfo
case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
val allReplicas: Set[Int])
包含,leaderIsrAndControllerEpoch,记录leader和isr
allReplicas记录所有的replicas,即AR,注意这里只会记录replica id,replica的具体情况,只会在replicaManager里面记录
这里为每个partition记录leaderIsrAndControllerEpoch,是不是有点浪费
而aliveBrokers,记录所有活的brokers的id和ip:port
所以也比较简单,这个cache在每个brokers之间是会被异步更新的,通过handleUpdateMetadataRequest
handleStopReplicaRequest
停止replica请求,一般是当broker stop或需要删除某replica时被调用
处理很简单,主要就是停止fetcher线程,并删除partition目录
stopReplicas
def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[TopicAndPartition, Short], Short) = {
replicaStateChangeLock synchronized { // 加锁
val responseMap = new collection.mutable.HashMap[TopicAndPartition, Short]
if(stopReplicaRequest.controllerEpoch < controllerEpoch) { // 检查Epoch,防止收到过期的request
(responseMap, ErrorMapping.StaleControllerEpochCode)
} else {
controllerEpoch = stopReplicaRequest.controllerEpoch // 更新Epoch
// First stop fetchers for all partitions, then stop the corresponding replicas
replicaFetcherManager.removeFetcherForPartitions(stopReplicaRequest.partitions.map(r => TopicAndPartition(r.topic, r.partition))) // 先通过FetcherManager停止相关partition的Fetcher线程
for(topicAndPartition <- stopReplicaRequest.partitions){
val errorCode = stopReplica(topicAndPartition.topic, topicAndPartition.partition, stopReplicaRequest.deletePartitions) // 调用stopReplica
responseMap.put(topicAndPartition, errorCode)
}
(responseMap, ErrorMapping.NoError)
}
}
}
stopReplica,注意很多情况下是不需要真正删除replica的,比如宕机
def stopReplica(topic: String, partitionId: Int, deletePartition: Boolean): Short = {
getPartition(topic, partitionId) match {
case Some(partition) =>
leaderPartitionsLock synchronized {
leaderPartitions -= partition
}
if(deletePartition) { // 仅仅在deletePartition=true时,才会真正删除该partition
val removedPartition = allPartitions.remove((topic, partitionId))
if (removedPartition != null)
removedPartition.delete() // this will delete the local log
}
case None => //do nothing if replica no longer exists. This can happen during delete topic retries
}
}
handleLeaderAndIsrRequest
处理leaderAndIsr的更新,这个和handleUpdateMetadataRequest的区别是,不光更新cache,需要真正去做replica的leader切换
主要调用,
replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager)
核心逻辑如下,前面那段主要是判断这个request是否有效,根据controllerEpoch和leaderEpoch
def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = {
replicaStateChangeLock synchronized {// 加锁
val responseMap = new collection.mutable.HashMap[(String, Int), Short]
if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { // 检查requset epoch
(responseMap, ErrorMapping.StaleControllerEpochCode)
} else {
val controllerId = leaderAndISRRequest.controllerId
val correlationId = leaderAndISRRequest.correlationId
controllerEpoch = leaderAndISRRequest.controllerEpoch // First check partition's leader epoch
// 前面只是检查了request的epoch,但是还要检查其中的每个partitionStateInfo中的leader epoch
val partitionState = new HashMap[Partition, PartitionStateInfo]()
leaderAndISRRequest.partitionStateInfos.foreach{ case ((topic, partitionId), partitionStateInfo) =>
val partition = getOrCreatePartition(topic, partitionId, partitionStateInfo.replicationFactor) // get或创建partition,partition只是逻辑存在,所以也是创建partition对象
val partitionLeaderEpoch = partition.getLeaderEpoch()
// If the leader epoch is valid record the epoch of the controller that made the leadership decision.
// This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path
if (partitionLeaderEpoch < partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch) { // local的partitionLeaderEpoch要小于request中的leaderEpoch,否则就是过时的request
if(partitionStateInfo.allReplicas.contains(config.brokerId)) // 判断该partition是否被assigned给当前的broker
partitionState.put(partition, partitionStateInfo) // 只将被分配到当前broker的partition放入partitionState,其中partition是当前的状况,partitionStateInfo是request中最新的状况
else { }
} else { // Received invalid LeaderAndIsr request
// Otherwise record the error code in response
responseMap.put((topic, partitionId), ErrorMapping.StaleLeaderEpochCode)
}
} //核心逻辑,判断是否为leader或follower,分别调用makeLeaders和makeFollowers
val partitionsTobeLeader = partitionState //从partitionState中筛选出以该broker为leader replica的
.filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId}
val partitionsToBeFollower = (partitionState -- partitionsTobeLeader.keys) if (!partitionsTobeLeader.isEmpty) makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap)
if (!partitionsToBeFollower.isEmpty) makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions
// have been completely populated before starting the checkpointing there by avoiding weird race conditions
if (!hwThreadInitialized) {
startHighWaterMarksCheckPointThread() // 启动HighWaterMarksCheckPointThread,hw很重要,需要定期存到磁盘,这样failover的时候可以重新load
hwThreadInitialized = true
}
replicaFetcherManager.shutdownIdleFetcherThreads() //关闭idle的fether,如果成为leader,就不需要fetch
(responseMap, ErrorMapping.NoError)
}
}
}
replicaManager里面有个allPartitions,记录所有partition的情况,
private val allPartitions = new Pool[(String, Int), Partition]
其中Partition结构中,比较主要的数据是,
private val assignedReplicaMap = new Pool[Int, Replica]
这个记录brokerid和replica的对应关系
def getOrCreatePartition(topic: String, partitionId: Int): Partition = {
var partition = allPartitions.get((topic, partitionId))
if (partition == null) {
allPartitions.putIfNotExists((topic, partitionId), new Partition(topic, partitionId, time, this))
partition = allPartitions.get((topic, partitionId))
}
partition
}
所以getOrCreatePartition,只是get当前replicaManager里面保存的该partiiton的情况
replicaManager.makeLeaders
关闭所有成为leader的replica对应的fetcher,然后关键是调用,
// Update the partition information to be the leader
partitionState.foreach{ case (partition, partitionStateInfo) =>
partition.makeLeader(controllerId, partitionStateInfo, correlationId)}
上面提到case (partition, partitionStateInfo)中,partition是replicaManager当前的情况,而partitionStateInfo中间放的是request的新的分配情况,
def makeLeader(controllerId: Int,
partitionStateInfo: PartitionStateInfo, correlationId: Int,
offsetManager: OffsetManager): Boolean = {
inWriteLock(leaderIsrUpdateLock) {
val allReplicas = partitionStateInfo.allReplicas
val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
// record the epoch of the controller that made the leadership decision. This is useful while updating the isr
// to maintain the decision maker controller's epoch in the zookeeper path
controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch
// add replicas that are new
allReplicas.foreach(replica => getOrCreateReplica(replica)) //request中allReplicas
val newInSyncReplicas = leaderAndIsr.isr.map(r => getOrCreateReplica(r)).toSet //request中ISR中的所有replicas
// remove assigned replicas that have been removed by the controller
// assignedReplicas表示当前partition分配情况,需要根据allReplicas更新,如果replicaid不在allReplicas中,则需要从assignedReplicas中删除
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_))
inSyncReplicas = newInSyncReplicas // 用request中的数据来更新当前partition中的
leaderEpoch = leaderAndIsr.leaderEpoch
zkVersion = leaderAndIsr.zkVersion
leaderReplicaIdOpt = Some(localBrokerId)
// construct the high watermark metadata for the new leader replica
val newLeaderReplica = getReplica().get
newLeaderReplica.convertHWToLocalOffsetMetadata() //对于新建的replica,只有offset,需要从log读取一下metadata
// reset log end offset for remote replicas
// 理解这,关键知道leo什么时候被更新的,leo只有当follower成功fetch leader的数据时,才会更新leader.assignedReplicas.getReplica.leo
// 所以这里需要把leo给reset,因为如果有数据,可能是上次该broker称为leader时的遗留数据
assignedReplicas.foreach(r => if (r.brokerId != localBrokerId) r.logEndOffset = LogOffsetMetadata.UnknownOffsetMetadata)
// 上面把所有remote replica的leo重置了成UnknownOffsetMetadata(-1),那么在maybeIncrementLeaderHW中会取所有replica中最小的leo,如果除leader外有其他replica,因为刚被重置过,最小leo一定是-1
// -1一定小于当前的hw,所以hw其实不会被increment。只有当isr中只有leader时,那hw会被increment到leader.leo
maybeIncrementLeaderHW(newLeaderReplica)
if (topic == OffsetManager.OffsetsTopicName)
offsetManager.loadOffsetsFromLog(partitionId)
true
}
}
这里还有个函数getOrCreateReplica,知道两点,
a. 在这里当local replica不存在的时候,会真正的创建replica
b. 所有生成replica都是用这个函数,所以其他的replica list都是assignedReplicaMap中replica的引用,比如inSyncReplicas
def getOrCreateReplica(replicaId: Int = localBrokerId): Replica = {
val replicaOpt = getReplica(replicaId)//assignedReplicaMap.get(replicaId)
replicaOpt match {
case Some(replica) => replica
case None =>
if (isReplicaLocal(replicaId)) { //如果是local,并且在AR中没有,那么需要创建这个replica
val config = LogConfig.fromProps(logManager.defaultConfig.toProps, AdminUtils.fetchTopicConfig(zkClient, topic))
val log = logManager.createLog(TopicAndPartition(topic, partitionId), config) //真正的创建replica文件
val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) //需要读出hw checkpoint
val offsetMap = checkpoint.read
if (!offsetMap.contains(TopicAndPartition(topic, partitionId)))
warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId))
val offset = offsetMap.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset) //读出hw,和loe求min,防止hw大于loe
val localReplica = new Replica(replicaId, this, time, offset, Some(log))
addReplicaIfNotExists(localReplica)
} else { //
val remoteReplica = new Replica(replicaId, this, time)
addReplicaIfNotExists(remoteReplica)
}
getReplica(replicaId).get
}
}
replicaManager.makeFollowers
var partitionsToMakeFollower: Set[Partition] = Set() //记录leader发生变化的partition
//调用partition.makeFollower
if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, offsetManager)) // 仅仅当partition的leader发生变化时才返回true,因为如果不变,不需要做任何操作
partitionsToMakeFollower += partition
//由于leader已发生变化,需要把旧的fetcher删除
replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(new TopicAndPartition(_))) //由于leader已发生变化,所以之前和旧leader同步的数据可能和新的leader是不一致的,但hw以下的数据,大家都是一致的,所以就把hw以上的数据truncate掉,防止不一致
logManager.truncateTo(partitionsToMakeFollower.map(partition => (new TopicAndPartition(partition), partition.getOrCreateReplica().highWatermark.messageOffset)).toMap) if (isShuttingDown.get()) {
//真正shuttingDown,就不要再加fetcher
}
else {
// we do not need to check if the leader exists again since this has been done at the beginning of this process
val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => //
new TopicAndPartition(partition) -> BrokerAndInitialOffset(
leaders.find(_.id == partition.leaderReplicaIdOpt.get).get,
partition.getReplica().get.logEndOffset.messageOffset)).toMap replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) //增加新的fetcher
}
partition.makeFollower
比较简单,只是更新assignedReplicas和ISR
def makeFollower(controllerId: Int,
partitionStateInfo: PartitionStateInfo,
correlationId: Int, offsetManager: OffsetManager): Boolean = {
inWriteLock(leaderIsrUpdateLock) {
val allReplicas = partitionStateInfo.allReplicas
val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
val newLeaderBrokerId: Int = leaderAndIsr.leader
// record the epoch of the controller that made the leadership decision. This is useful while updating the isr
// to maintain the decision maker controller's epoch in the zookeeper path
controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch
// add replicas that are new
allReplicas.foreach(r => getOrCreateReplica(r))
// remove assigned replicas that have been removed by the controller
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_))
inSyncReplicas = Set.empty[Replica] // 将isr置空,不同于makeLeader
leaderEpoch = leaderAndIsr.leaderEpoch
zkVersion = leaderAndIsr.zkVersion if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { // 判断replica leader是否发生了变化
false
}
else {
leaderReplicaIdOpt = Some(newLeaderBrokerId) // 如果发生变化,则更新leader
true
}
}
}
Apache Kafka源码分析 - KafkaApis的更多相关文章
-
Apache Kafka源码分析 &ndash; Broker Server
1. Kafka.scala 在Kafka的main入口中startup KafkaServerStartable, 而KafkaServerStartable这是对KafkaServer的封装 1: ...
-
apache kafka源码分析-Producer分析---转载
原文地址:http://www.aboutyun.com/thread-9938-1-1.html 问题导读1.Kafka提供了Producer类作为java producer的api,此类有几种发送 ...
-
Apache Kafka源码分析 - kafka controller
前面已经分析过kafka server的启动过程,以及server所能处理的所有的request,即KafkaApis 剩下的,其实关键就是controller,以及partition和replica ...
-
Apache Kafka源码分析 &ndash; Controller
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Controller+Internalshttps://cwiki.apache.org ...
-
Apache Kafka源码分析 &ndash; Log Management
LogManager LogManager会管理broker上所有的logs(在一个log目录下),一个topic的一个partition对应于一个log(一个log子目录)首先loadLogs会加载 ...
-
Apache Kafka源码分析 - autoLeaderRebalanceEnable
在broker的配置中,auto.leader.rebalance.enable (false) 那么这个leader是如何进行rebalance的? 首先在controller启动的时候会打开一个s ...
-
Apache Kafka源码分析 &ndash; Replica and Partition
Replica 对于local replica, 需要记录highWatermarkValue,表示当前已经committed的数据对于remote replica,需要记录logEndOffsetV ...
-
Apache Kafka源码分析 &ndash; ReplicaManager
如果说controller作为master,负责全局的事情,比如选取leader,reassignment等那么ReplicaManager就是worker,负责完成replica的管理工作 主要工作 ...
-
Apache Kafka源码分析 - ReplicaStateMachine
startup 在onControllerFailover中被调用, /** * Invoked on successful controller election. First registers ...
随机推荐
-
AJAX-----10iframe模拟ajax文件上传效果原理2
在实际开发中其实我们可以给用户一些提示,比如上传成功或者上传失败,废话不多说,走码: <!DOCTYPE html> <html lang="en"> &l ...
-
求n*m网格内矩形的数目
一个n*m的网格,求这个网格中矩形的数目. 比如以下2*2网格,总共有9个矩形:4个1*1的矩形,4个1*2的矩形,1个2*2的矩形 算法1:动态规划,假设dp[i][j]表示以第 i 行第 j ...
-
【NS2仿真】TCP与UDP混合
# # ftp # \ # tcp # \ # n0 sink # \ / # \ / # n1---5M 2ms---n3 # / \ # / \ # n2 null # / # udp # / # ...
-
定制你的Unity编辑器
Unity的编辑器可以通过写脚本进行界面定制,包括添加功能菜单,今天写游戏Demo用到了记录一下. 为Unity添加子菜单 示例程序 [AddComponentMenu("Defend Ho ...
-
关于JavaScripting API您不知道的5件事
现在,许多 Java 开发人员都喜欢在 Java 平台中使用脚本语言,但是使用编译到 Java 字节码中的动态语言有时是不可行的.在某些情况中,直接编写一个 Java 应用程序的脚本 部分 或者在一个 ...
-
组件接口(API)设计指南[5]-最后的思考
*阅读其它章节: http://blog.csdn.net/cuibo1123/article/details/39894477 最后的思考 我通过困难的学习以及多年的失误.写了这片篇关于创建组件和a ...
-
PHP 领域模型与数据库映射文章
1. http://blog.csdn.net/happen_zhang/article/details/12761747 2. http://blog.csdn.net/hguisu/article ...
-
iOS - 提示用户升级版本并跳转到AppStore
一.问题:自己做提示用户升级? 由于苹果做了自动升级,所有只要在应用程序中出现从AppStore检查版本更新,或者出现任何有关升级的提醒都会被拒,但是如果必须添加升级提示的话,可以配合后台通过添加AP ...
-
PHP webservice初探
背景:在最近的开发中,为了解决公司内部系统与外部系统的对接,开始接触到了webservice接口,外部公司提供接口供我们调用,已达到数据同步的目的,因此有必要普及一下web service的知识了! ...
-
C语言存储类型
看c专家编程,有说存储类型一直不太清楚.看到一篇文章讲解c的存储类型,讲解了c语言中的各种变量的存储类型,而且是从进程.内存的角度讲解的,以前从没有这样理解过,觉得挺有用的,在这里转载过来. 首先要来 ...