Follower 副本从 Leader 副本拉取数据是在ReplicaFetcherThread 线程完成的,现在先看看他的抽象基类AbstractFetcherThread的定义。
abstract class AbstractFetcherThread(name: String, // 线程名称
clientId: String, // Client Id,用于日志输出
val sourceBroker: BrokerEndPoint, // 数据源Broker地址,即分区的leader是在哪个节点
failedPartitions: FailedPartitions,// 处理过程中出现失败的分区
fetchBackOffMs: Int = 0,// 当获取分区数据出错后的等待重试间隔,默认是 Broker 端参数 replica.fetch.backoff.ms 值。
isInterruptible: Boolean = true)// 线程是否允许被中断
extends ShutdownableThread(name, isInterruptible) {
// 定义FetchData类型表示获取的消息数据
type FetchData = FetchResponse.PartitionData[Records]
// 定义EpochData类型表示Leader Epoch数据
type EpochData = OffsetsForLeaderEpochRequest.PartitionData
// 源码会不断地调整那些不需要推迟的分区的读取顺序,以保证读取的公平性。
// 这个公平性,其实就是在 partitionStates 字段的类型 PartitionStates 类中实现的。这个类是在 clients 工程中定义的。
// 它本质上会接收一组要读取的主题分区,然后以轮询的方式依次读取这些分区以确保公平性。
// PartitionStates里面的moveToEnd方法,就是把当前的拉的分区,放到最后一个,保证顺序公平性
private val partitionStates = new PartitionStates[PartitionFetchState]
}
其中FetchData 定义里的 PartitionData 类型,是客户端 clients 工程中 FetchResponse 类定义的嵌套类。FetchResponse 类封装的是 FETCH 请求的 Response 对象,而里面的 PartitionData 类保存的是 Response 中单个分区数据拉取的各项数据,包括从该分区的 Leader 副本拉取回来的消息、该分区的高水位值和日志起始位移值等。
public static final class PartitionData<T extends BaseRecords> {
public final Errors error; // 错误码
public final long highWatermark;// 高水位值
public final long lastStableOffset;// 最新LSO值
public final long logStartOffset;// 最新Log Start Offset值
public final Optional<Integer> preferredReadReplica; // 用于指定可对外提供读服务的 Follower 副本;KAFKA 2.4之后支持部分Follower副本可以对外提供读服务
public final List<AbortedTransaction> abortedTransactions; // 该分区对应的已终止事务列表
public final T records; // 消息集合,最重要的字段!
}
其中的PartitionStates[PartitionFetchState]表征分区读取状态的,保存的是分区的已读取位移值和对应的副本状态。它的定义为:
case class PartitionFetchState(fetchOffset: Long,
currentLeaderEpoch: Int,
delay: DelayedItem,
state: ReplicaState) {
// 分区读取状态有 3 个,分别是:
// 可获取,表明副本获取线程当前能够读取数据。
def isReadyForFetch: Boolean = state == Fetching && !isDelayed
// 截断中,表明分区副本正在执行截断操作(比如该副本刚刚成为 Follower 副本)。
def isTruncating: Boolean = state == Truncating && !isDelayed
// 被推迟,表明副本获取线程获取数据时出现错误,需要等待一段时间后重试。
def isDelayed: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) > 0
}
在PartitionStates中,有一个重要的函数是不断地调整那些不需要推迟的分区的读取顺序,以保证读取的公平性。实现就是吧队列头放到队列尾巴。
public void updateAndMoveToEnd(TopicPartition topicPartition, S state) {
map.remove(topicPartition);
map.put(topicPartition, state);
updateSize();
}
下面介绍AbstractFetcherThread的几个重要方法
processPartitionData方法
用于处理读取回来的消息集合。processPartitionData是一个抽象方法,因此需要子类实现它的逻辑
protected def processPartitionData( topicPartition: TopicPartition, // 读取哪个分区的数据
fetchOffset: Long, // 读取到的最新位移值
partitionData: FetchData // 读取到的分区消息数据
): Option[LogAppendInfo] // 写入已读取消息数据前的元数据
truncate 方法
截取消息,需要子类实现它的逻辑
protected def truncate(
topicPartition: TopicPartition, // 要对哪个分区下副本执行截断操作
truncationState: OffsetTruncationState // Offset + 截断状态
): Unit
buildFetch 方法
为指定分区构建对应的 FetchRequest.Builder 对象,而该对象是构建 FetchRequest 的核心组件。Kafka 中任何类型的消息读取,都是通过给指定 Broker 发送 FetchRequest 请求来完成的。需要子类实现它的逻辑。
protected def buildFetch( // 一组要读取的分区列表
// 分区是否可读取取决于PartitionFetchState中的状态
partitionMap: Map[TopicPartition, PartitionFetchState]):
// 封装FetchRequest.Builder对象
ResultWithPartitions[Option[FetchRequest.Builder]]
doWork 线程主方法
override def doWork(): Unit = {
maybeTruncate() // 执行副本截断操作
maybeFetch() // 执行消息获取操作
}
maybeTruncate()执行阶段操作。AbstractFetcherThread 线程总要不断尝试去做截断的原因是,分区的 Leader 可能会随时发生变化。每当有新 Leader 产生时,Follower 副本就必须主动执行截断操作,将自己的本地日志裁剪成与 Leader 一模一样的消息序列,甚至,Leader 副本本身也需要执行截断操作,将 LEO 调整到分区高水位处。
private def maybeTruncate(): Unit = {
// 将所有处于截断中状态的分区依据有无Leader Epoch值进行分组
val (partitionsWithEpochs, partitionsWithoutEpochs) = fetchTruncatingPartitions()
// 对于有Leader Epoch值的分区,将日志截断到Leader Epoch值对应的位移值处
if (partitionsWithEpochs.nonEmpty) {
truncateToEpochEndOffsets(partitionsWithEpochs)
}
// 对于没有Leader Epoch值的分区,将日志截断到高水位值处
if (partitionsWithoutEpochs.nonEmpty) {
truncateToHighWatermark(partitionsWithoutEpochs)
}
}
其中truncateToHighWatermark方法如下,高水位值在截断操作中的应用。
private[server] def truncateToHighWatermark(partitions: Set[TopicPartition]): Unit = inLock(partitionMapLock) {
val fetchOffsets = mutable.HashMap.empty[TopicPartition, OffsetTruncationState]
// 遍历每个要执行截断操作的分区对象
for (tp <- partitions) {
// 获取分区的分区读取状态
val partitionState = partitionStates.stateValue(tp)
if (partitionState != null) {
// 取出高水位值。分区的最大可读取位移值就是高水位值
val highWatermark = partitionState.fetchOffset
val truncationState = OffsetTruncationState(highWatermark, truncationCompleted = true)
info(s"Truncating partition $tp to local high watermark $highWatermark")
// 执行截断到高水位值
if (doTruncate(tp, truncationState))
fetchOffsets.put(tp, truncationState)
}
}
// 更新这组分区的分区读取状态
updateFetchOffsetAndMaybeMarkTruncationComplete(fetchOffsets)
}
maybeFetch方法从leader中获取消息并处理。
private def maybeFetch(): Unit = {
val (fetchStates, fetchRequestOpt) = inLock(partitionMapLock) {
// 为partitionStates中的分区构造FetchRequest
// partitionStates中保存的是要去获取消息的分区以及对应的状态
val fetchStates = partitionStates.partitionStateMap.asScala
val ResultWithPartitions(fetchRequestOpt, partitionsWithError) = buildFetch(fetchStates)
// 处理出错的分区,处理方式主要是将这个分区加入到有序Map末尾
// 等待后续重试
handlePartitionsWithErrors(partitionsWithError, "maybeFetch")
// 如果当前没有可读取的分区,则等待fetchBackOffMs时间等候后续重试
if (fetchRequestOpt.isEmpty) {
trace(s"There are no active partitions. Back off for $fetchBackOffMs ms before sending a fetch request")
partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS)
}
(fetchStates, fetchRequestOpt)
}
// 发送FETCH请求给Leader副本,并处理Response
fetchRequestOpt.foreach { fetchRequest =>
processFetchRequest(fetchStates, fetchRequest)
}
}
其中processFetchRequest方法发送拉取消息请求并处理收到的消息。
private def processFetchRequest(fetchStates: Map[TopicPartition, PartitionFetchState],
fetchRequest: FetchRequest.Builder): Unit = {
val partitionsWithError = mutable.Set[TopicPartition]()
var responseData: Seq[(TopicPartition, FetchData)] = Seq.empty
try {
trace(s"Sending fetch request $fetchRequest")
// 给Leader发送FETCH请求
responseData = fetchFromLeader(fetchRequest)
} catch {
case t: Throwable =>
if (isRunning) {
warn(s"Error in response for fetch request $fetchRequest", t)
inLock(partitionMapLock) {
partitionsWithError ++= partitionStates.partitionSet.asScala
// there is an error occurred while fetching partitions, sleep a while
// note that `ReplicaFetcherThread.handlePartitionsWithError` will also introduce the same delay for every
// partition with error effectively doubling the delay. It would be good to improve this.
partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS)
}
}
}
// 更新请求发送速率指标
fetcherStats.requestRate.mark()
// 处理响应
if (responseData.nonEmpty) {
// process fetched data
inLock(partitionMapLock) {
responseData.foreach { case (topicPartition, partitionData) =>
Option(partitionStates.stateValue(topicPartition)).foreach { currentFetchState =>
// It's possible that a partition is removed and re-added or truncated when there is a pending fetch request.
// In this case, we only want to process the fetch response if the partition state is ready for fetch and
// the current offset is the same as the offset requested.
// 获取分区核心信息
val fetchState = fetchStates(topicPartition)
// 处理Response的条件:
// 1. fetch请求中要获取的位移值和之前已保存的下一条待获取位移值相等
// 2. 当前分区处于可获取状态
if (fetchState.fetchOffset == currentFetchState.fetchOffset && currentFetchState.isReadyForFetch) {
// 提取Response中的Leader Epoch值
val requestEpoch = if (fetchState.currentLeaderEpoch >= 0)
Some(fetchState.currentLeaderEpoch)
else
None
partitionData.error match {
// 如果没有错误
case Errors.NONE =>
try {
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
// 交由子类完成Response的处理,即给ReplicaFetcherThread的processPartitionData处理,这个稍后会介绍
val logAppendInfoOpt = processPartitionData(topicPartition, currentFetchState.fetchOffset,
partitionData)
logAppendInfoOpt.foreach { logAppendInfo =>
val validBytes = logAppendInfo.validBytes
// 获取返回的最后一条消息的 offset 值
val nextOffset = if (validBytes > 0) logAppendInfo.lastOffset + 1 else currentFetchState.fetchOffset
// 更新lag
fetcherLagStats.getAndMaybePut(topicPartition).lag = Math.max(0L, partitionData.highWatermark - nextOffset)
// ReplicaDirAlterThread may have removed topicPartition from the partitionStates after processing the partition data
if (validBytes > 0 && partitionStates.contains(topicPartition)) {
// Update partitionStates only if there is no exception during processPartitionData
val newFetchState = PartitionFetchState(nextOffset, fetchState.currentLeaderEpoch,
state = Fetching)
// 将该分区放置在有序Map读取顺序的末尾,保证公平性
partitionStates.updateAndMoveToEnd(topicPartition, newFetchState)
fetcherStats.byteRate.mark(validBytes)
}
}
} catch {
case ime@( _: CorruptRecordException | _: InvalidRecordException) =>
// we log the error and continue. This ensures two things
// 1. If there is a corrupt message in a topic partition, it does not bring the fetcher thread
// down and cause other topic partition to also lag
// 2. If the message is corrupt due to a transient state in the log (truncation, partial writes
// can cause this), we simply continue and should get fixed in the subsequent fetches
error(s"Found invalid messages during fetch for partition $topicPartition " +
s"offset ${currentFetchState.fetchOffset}", ime)
partitionsWithError += topicPartition
case e: KafkaStorageException =>
error(s"Error while processing data for partition $topicPartition " +
s"at offset ${currentFetchState.fetchOffset}", e)
markPartitionFailed(topicPartition)
case t: Throwable =>
// stop monitoring this partition and add it to the set of failed partitions
error(s"Unexpected error occurred while processing data for partition $topicPartition " +
s"at offset ${currentFetchState.fetchOffset}", t)
markPartitionFailed(topicPartition)
}
// 如果读取位移值越界,通常是因为Leader发生变更
case Errors.OFFSET_OUT_OF_RANGE =>
// 调整越界,主要办法是做截断
if (handleOutOfRangeError(topicPartition, currentFetchState, requestEpoch))
// 如果依然不能成功,加入到出错分区列表
partitionsWithError += topicPartition
// 如果Leader Epoch值比Leader所在Broker上的Epoch值要新
case Errors.UNKNOWN_LEADER_EPOCH =>
debug(s"Remote broker has a smaller leader epoch for partition $topicPartition than " +
s"this replica's current leader epoch of ${fetchState.currentLeaderEpoch}.")
// 加入到出错分区列表
partitionsWithError += topicPartition
// 如果Leader Epoch值比Leader所在Broker上的Epoch值要旧
case Errors.FENCED_LEADER_EPOCH =>
if (onPartitionFenced(topicPartition, requestEpoch)) partitionsWithError += topicPartition
// 如果Leader发生变更
case Errors.NOT_LEADER_FOR_PARTITION =>
debug(s"Remote broker is not the leader for partition $topicPartition, which could indicate " +
"that the partition is being moved")
// 加入到出错分区列表
partitionsWithError += topicPartition
case _ =>
error(s"Error for partition $topicPartition at offset ${currentFetchState.fetchOffset}",
partitionData.error.exception)
// 加入到出错分区列表
partitionsWithError += topicPartition
}
}
}
}
}
}
if (partitionsWithError.nonEmpty) {
// 处理出错分区列表
// 对于操作存在异常的 topic 分区,暂停发送 FetchRequest 请求,休息一会儿
handlePartitionsWithErrors(partitionsWithError, "processFetchRequest")
}
}
其中当出现越界时,调用handleOutOfRangeError方法,此方法主要是调用fetchOffsetAndTruncate进行截断,最终调用子类ReplicaFetcherThread的truncate 方法
protected def fetchOffsetAndTruncate(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = {
val replicaEndOffset = logEndOffset(topicPartition)
/**
* Unclean leader election: A follower goes down, in the meanwhile the leader keeps appending messages. The follower comes back up
* and before it has completely caught up with the leader's logs, all replicas in the ISR go down. The follower is now uncleanly
* elected as the new leader, and it starts appending messages from the client. The old leader comes back up, becomes a follower
* and it may discover that the current leader's end offset is behind its own end offset.
*
* In such a case, truncate the current follower's log to the current leader's end offset and continue fetching.
*
* There is a potential for a mismatch between the logs of the two replicas here. We don't fix this mismatch as of now.
*/
// 发送 ListOffsetRequest 请求,获取 leader 副本的 LEO 值
val leaderEndOffset = fetchLatestOffsetFromLeader(topicPartition, currentLeaderEpoch)
// 如果 leader 副本的 LEO 值落后于 follower 副本的 LEO 值
if (leaderEndOffset < replicaEndOffset) {
warn(s"Reset fetch offset for partition $topicPartition from $replicaEndOffset to current " +
s"leader's latest offset $leaderEndOffset")
// 将分区对应的 Log 截断到 leader 副本的 LEO 位置,从该位置开始重新与 leader 副本进行同步
truncate(topicPartition, OffsetTruncationState(leaderEndOffset, truncationCompleted = true))
leaderEndOffset
} else {
// 可能场景:follower 副本长时间失效,期间 leader 副本不断在追加新的数据,等到 follower 再次上线时,leader 副本对应 offset 位置的日志数据已被定时任务清除。
/**
* If the leader's log end offset is greater than the follower's log end offset, there are two possibilities:
* 1. The follower could have been down for a long time and when it starts up, its end offset could be smaller than the leader's
* start offset because the leader has deleted old logs (log.logEndOffset < leaderStartOffset).
* 2. When unclean leader election occurs, it is possible that the old leader's high watermark is greater than
* the new leader's log end offset. So when the old leader truncates its offset to its high watermark and starts
* to fetch from the new leader, an OffsetOutOfRangeException will be thrown. After that some more messages are
* produced to the new leader. While the old leader is trying to handle the OffsetOutOfRangeException and query
* the log end offset of the new leader, the new leader's log end offset becomes higher than the follower's log end offset.
*
* In the first case, the follower's current log end offset is smaller than the leader's log start offset. So the
* follower should truncate all its logs, roll out a new segment and start to fetch from the current leader's log
* start offset.
* In the second case, the follower should just keep the current log segments and retry the fetch. In the second
* case, there will be some inconsistency of data between old and new leader. We are not solving it here.
* If users want to have strong consistency guarantees, appropriate configurations needs to be set for both
* brokers and producers.
*
* Putting the two cases together, the follower should fetch from the higher one of its replica log end offset
* and the current leader's log start offset.
*/
// 发送 ListOffsetRequest 请求,获取 leader 副本的 startOffset 值
val leaderStartOffset = fetchEarliestOffsetFromLeader(topicPartition, currentLeaderEpoch)
warn(s"Reset fetch offset for partition $topicPartition from $replicaEndOffset to current " +
s"leader's start offset $leaderStartOffset")
// 选择下次获取消息的起始 offset 值
val offsetToFetch = Math.max(leaderStartOffset, replicaEndOffset)
// Only truncate log when current leader's log start offset is greater than follower's log end offset.
// 如果当前 leader 的 startOffset 大于对应副本的 LEO 值,则将该副本的 Log 全部截断,并创建新的 activeSegment 对象
if (leaderStartOffset > replicaEndOffset)
truncateFullyAndStartAt(topicPartition, leaderStartOffset)
// 返回下次获取消息的 offset 位置
offsetToFetch
}
}