ISR列表是如何變化的?

  • ISR列表: 所有同partiton leader數據同步的Replica集合;
  • 在不允許partition leader臟選舉的情況下, partition leader只能從ISR列表中選取;
  • 根據ISR的定義可知, ISR列表的成員是有可能動態變化的, 集合可能被擴充, 也可能被收縮;
  • ISR列表的維護由每個Partition的leader replica負責;

ISR列表收縮
  • ReplicatManager在啟動時會啟動一個周期性任務, 來定期查看是否有ISR列表需要收縮: scheduler.schedule("isr-expiration", maybeShrinkIsr, period = config.replicaLagTimeMaxMs, unit = TimeUnit.MILLISECONDS), 這個操作針對每個partition都進行檢查, 最后會調用Partition::maybeShrinkIsr:
     val leaderHWIncremented = inWriteLock(leaderIsrUpdateLock) {
      leaderReplicaIfLocal() match {
        case Some(leaderReplica) =>
          val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs)
          if(outOfSyncReplicas.size > 0) {
            val newInSyncReplicas = inSyncReplicas -- outOfSyncReplicas
            assert(newInSyncReplicas.size > 0)
            info("Shrinking 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 in cache
            updateIsr(newInSyncReplicas)
            // we may need to increment high watermark since ISR could be down to 1

            replicaManager.isrShrinkRate.mark()
            maybeIncrementLeaderHW(leaderReplica) // ? 如果更新了HighWaterMark, 是否也要調用tryCompleteDelayedRequests()???
          } else {
            false
          }

        case None => false // do nothing if no longer leader
      }

  1. 核心是調用getOutOfSyncReplicas得到當前沒有同步跟上leader的Replicat列表, 然后從inSyncReplicas中踢除掉后更新本地的metadata ISR緩存同時更新zk上/brokers/topics/[topic]/partitions/[parition]/stat的節點內容, 最后因為ISR列表成員減少了, 需要重新評估是否需要更新leaderhigh water mark;
  2. getOutOfSyncReplicas: 得到當前沒有同步跟上leader的Replicat列表
    /**
     * there are two cases that will be handled here -
     * 1. Stuck followers: If the leo of the replica hasn't been updated for maxLagMs ms,
     *                     the follower is stuck and should be removed from the ISR
     * 2. Slow followers: If the replica has not read up to the leo within the last maxLagMs ms,
     *                    then the follower is lagging and should be removed from the ISR
     * Both these cases are handled by checking the lastCaughtUpTimeMs which represents
     * the last time when the replica was fully caught up. If either of the above conditions
     * is violated, that replica is considered to be out of sync
     *
     **/
    val leaderLogEndOffset = leaderReplica.logEndOffset
    val candidateReplicas = inSyncReplicas - leaderReplica

    val laggingReplicas = candidateReplicas.filter(r => (time.milliseconds - r.lastCaughtUpTimeMs) > maxLagMs)
    if(laggingReplicas.size > 0)
      debug("Lagging replicas for partition %s are %s".format(TopicAndPartition(topic, partitionId), laggingReplicas.map(_.brokerId).mkString(",")))

    laggingReplicas

源碼中的注釋已經寫得很清楚了.

  1. 被淘汰后ISR列表的條件是(time.milliseconds - replicat.lastCaughtUpTimeMs) > maxLagMs
  2. replicat.lastCaughtUpTimeMs何時被更新呢? 其實是 Replica::updateLogResult中:
   def updateLogReadResult(logReadResult : LogReadResult) {
    logEndOffset = logReadResult.info.fetchOffsetMetadata

    /* If the request read up to the log end offset snapshot when the read was initiated,
     * set the lastCaughtUpTimeMsUnderlying to the current time.
     * This means that the replica is fully caught up.
     */
    if(logReadResult.isReadFromLogEnd) {
      lastCaughtUpTimeMsUnderlying.set(time.milliseconds)
    }
  }
  1. 順藤摸瓜,會發現在響應FetchRequest請求時即ReplicaManager::fetchMessage中的updateFollowerLogReadResults(replicaId, logReadResults)會調用 Replica::updateLogResult, 當處理當前的FetchRequest請求時,如果已經讀取到了相應partiton leader的LogEndOffset了, 則可以更新lastCaughtUpTimeMsUnderlying, 表明當前的復本在這個FetchRequest請求返回后就進行同步跟上了leader的步伐;
  2. 有關響應FetchRequest請求的具體分析可參考Kafka是如何處理客戶端發送的數據的?
ISR列表擴容
  • ISR擴容操作位于Partition::maybeExpandIsr中:
val leaderHWIncremented = inWriteLock(leaderIsrUpdateLock) {
      // check if this replica needs to be added to the ISR
      leaderReplicaIfLocal() match {
        case Some(leaderReplica) =>
          val replica = getReplica(replicaId).get
          val leaderHW = leaderReplica.highWatermark
          if(!inSyncReplicas.contains(replica) &&
             assignedReplicas.map(_.brokerId).contains(replicaId) &&
                  replica.logEndOffset.offsetDiff(leaderHW) >= 0) {
            val newInSyncReplicas = inSyncReplicas + replica
            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)
            replicaManager.isrExpandRate.mark()
          }

          // check if the HW of the partition can now be incremented
          // since the replica maybe now be in the ISR and its LEO has just incremented
          maybeIncrementLeaderHW(leaderReplica)

        case None => false // nothing to do if no longer leader
      }
    }

    // some delayed operations may be unblocked after HW changed
    if (leaderHWIncremented)
      tryCompleteDelayedRequests()

核心replica.logEndOffset.offsetDiff(leaderHW) >= 0 如果當前replicaLEO大于等于LeaderHighWaterMark, 則表明該replica的同步已經跟上了leader, 將其加入到ISR列表中,更新本地的metadata ISR緩存同時更新zk上/brokers/topics/[topic]/partitions/[parition]/stat的節點內容;

  • Partition::maybeExpandIsr的調用時機: 在Replica::updateReplicaLogReadResult中被調用, 同樣順藤摸瓜,會發現也是在響應FetchRequest請求時即ReplicaManager::fetchMessage中的updateFollowerLogReadResults(replicaId, logReadResults)會調用;
ISR列表變化后, 更新集群內每臺broker上的metadata
  • 在上面的ISR列表收縮和擴容的同時,都會通過ReplicaManager::recordIsrChange來記錄有變化的 TopicAndParition;
  • ReplicaManager在啟動時還會啟動一個周期性任務maybePropagateIsrChanges, 來定期將ISR在變化的TopicAndParition信息寫入zk的/isr_change_notification節點;
  • KafkaController會監控zk的/isr_change_notification節點變化, 向所有的broker發送MetadataRequest;
  • 我們來看看maybePropagateIsrChanges的實現:
  val now = System.currentTimeMillis()
    isrChangeSet synchronized {
      if (isrChangeSet.nonEmpty &&
        (lastIsrChangeMs.get() + ReplicaManager.IsrChangePropagationBlackOut < now ||
          lastIsrPropagationMs.get() + ReplicaManager.IsrChangePropagationInterval < now)) {
        ReplicationUtils.propagateIsrChanges(zkUtils, isrChangeSet)
        isrChangeSet.clear()
        lastIsrPropagationMs.set(now)
      }
    }

可以看到為了防止將頻繁的ISR變化廣播到整個集群, 這里作了限制;

Kafka源碼分析-匯總

最后編輯于
?著作權歸作者所有,轉載或內容合作請聯系作者
平臺聲明:文章內容(如有圖片或視頻亦包括在內)由作者上傳并發布,文章內容僅代表作者本人觀點,簡書系信息發布平臺,僅提供信息存儲服務。

推薦閱讀更多精彩內容