概述
follower副本向leader副本发送OffsetForLeaderEpochRequest后,leader副本返回大于requestedLeaderEpoch的最小Epoch的startOffset。
follower副本会根据返回的leaderEpoch和offset进行日志截断。
首先,我将通过一个例子来说明follower副本向leader副本请求OffsetForLeaderEpoch的过程
A(leader, epoch=1): 1, 2, 3, 4, 5, 6
A cache: leaderEpoch = 1, startOffset = 1
B(follower): 1, 2, 3, 4
B cache: leaderEpoch = 1, startOffset = 1
=============================================
B(leader, epoch=2): 1, 2, 3, 4, 5, 6, 7
B cache:
leaderEpoch = 1, startOffset = 1
leaderEpoch = 2, startOffset = 5、
A挂掉后,B成为新leader,A又恢复过来,此时追加了新数据,B的leaderEpochCache增加了新条目(leaderEpoch=2, startOffset=5)。
当A请求复制B时,请求的epoch为1,B查询到epoch=2(比1大的最小epoch),然后返回对应的startOffset=5,A收到后truncate自己>=5的记录(这里是offset=5和6),然后把请求的offset更新为5,重新复制数据,B返回数据(offset=5, 6 和7,epoch=2),A追加记录时发现数据的epoch=2,新增条目(epoch=2, startOffset=5)到自己的leaderEpochCache。
下面将通过源码分析follower副本会根据返回的leaderEpoch和offset进行日志截断的过程。
AbstractFetcherThread#maybeTruncate方法
private def maybeTruncate(): Unit = {
val (partitionsWithEpochs, partitionsWithoutEpochs) = fetchTruncatingPartitions()
if (partitionsWithEpochs.nonEmpty) {
truncateToEpochEndOffsets(partitionsWithEpochs)
}
if (partitionsWithoutEpochs.nonEmpty) {
truncateToHighWatermark(partitionsWithoutEpochs)
}
}
AbstractFetcherThread#truncateToEpochEndOffsets方法
向leader副本发送OffsetsForLeaderEpochRequest,返回每个Partition对应的EpochEndOffset。
将每个partition的日志截断到指定的offset处。截断的整个过程需要加锁保护,以防leadership发生改变。
/**
* - Build a leader epoch fetch based on partitions that are in the Truncating phase
* - Send OffsetsForLeaderEpochRequest, retrieving the latest offset for each partition's
* leader epoch. This is the offset the follower should truncate to ensure
* accurate log replication.
* - Finally truncate the logs for partitions in the truncating phase and mark them
* truncation complete. Do this within a lock to ensure no leadership changes can
* occur during truncation.
*/
private def truncateToEpochEndOffsets(latestEpochsForPartitions: Map[TopicPartition, EpochData]): Unit = {
//向leader副本发送OffsetsForLeaderEpochRequest,返回Map[TopicPartition, EpochEndOffset]集合
val endOffsets = fetchEpochEndOffsets(latestEpochsForPartitions)
//Ensure we hold a lock during truncation.
inLock(partitionMapLock) {
//Check no leadership and no leader epoch changes happened whilst we were unlocked, fetching epochs
//过滤map[TopicPartition, EpochEndOffset]集合,校验在发送OffsetsForLeaderEpochRequest到返回的过程中,leader replica没有发生改变
//得到过滤后的map[TopicPartition, EpochEndOffset]集合
val epochEndOffsets = endOffsets.filter { case (tp, _) =>
val curPartitionState = partitionStates.stateValue(tp)
val partitionEpochRequest = latestEpochsForPartitions.get(tp).getOrElse {
throw new IllegalStateException(
s"Leader replied with partition $tp not requested in OffsetsForLeaderEpoch request")
}
val leaderEpochInRequest = partitionEpochRequest.currentLeaderEpoch.get
curPartitionState != null && leaderEpochInRequest == curPartitionState.currentLeaderEpoch
}
val ResultWithPartitions(fetchOffsets, partitionsWithError) = maybeTruncateToEpochEndOffsets(epochEndOffsets)
handlePartitionsWithErrors(partitionsWithError)
//最后标记截断完成
updateFetchOffsetAndMaybeMarkTruncationComplete(fetchOffsets)
}
}
AbstractFetcherThread#maybeTruncateToEpochEndOffsets方法
private def maybeTruncateToEpochEndOffsets(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): ResultWithPartitions[Map[TopicPartition, OffsetTruncationState]] = {
val fetchOffsets = mutable.HashMap.empty[TopicPartition, OffsetTruncationState]
val partitionsWithError = mutable.HashSet.empty[TopicPartition]
fetchedEpochs.foreach { case (tp, leaderEpochOffset) =>
try {
leaderEpochOffset.error match {
case Errors.NONE =>
val offsetTruncationState = getOffsetTruncationState(tp, leaderEpochOffset)
truncate(tp, offsetTruncationState)
fetchOffsets.put(tp, offsetTruncationState)
case Errors.FENCED_LEADER_EPOCH =>
onPartitionFenced(tp)
case error =>
info(s"Retrying leaderEpoch request for partition $tp as the leader reported an error: $error")
partitionsWithError += tp
}
} catch {
case e: KafkaStorageException =>
info(s"Failed to truncate $tp", e)
partitionsWithError += tp
}
}
ResultWithPartitions(fetchOffsets, partitionsWithError)
}
ReplicaFetcherThread#truncate方法
/**
* Truncate the log for each partition's epoch based on leader's returned epoch and offset.
* The logic for finding the truncation offset is implemented in AbstractFetcherThread.getOffsetTruncationState
*/
override def truncate(tp: TopicPartition, offsetTruncationState: OffsetTruncationState): Unit = {
val replica = replicaMgr.localReplicaOrException(tp)
val partition = replicaMgr.getPartition(tp).get
partition.truncateTo(offsetTruncationState.offset, isFuture = false)
if (offsetTruncationState.offset < replica.highWatermark.messageOffset)
warn(s"Truncating $tp to offset ${offsetTruncationState.offset} below high watermark " +
s"${replica.highWatermark.messageOffset}")
// mark the future replica for truncation only when we do last truncation
if (offsetTruncationState.truncationCompleted)
replicaMgr.replicaAlterLogDirsManager.markPartitionsForTruncation(brokerConfig.brokerId, tp,
offsetTruncationState.offset)
}
Partition#truncateTo方法
基于给定的offset,截断该Partition的local log,并设置该offset为checkpoint。
/**
* Truncate the local log of this partition to the specified offset and checkpoint the recovery point to this offset
*
* @param offset offset to be used for truncation
* @param isFuture True iff the truncation should be performed on the future log of this partition
*/
def truncateTo(offset: Long, isFuture: Boolean) {
// The read lock is needed to prevent the follower replica from being truncated while ReplicaAlterDirThread
// is executing maybeDeleteAndSwapFutureReplica() to replace follower replica with the future replica.
inReadLock(leaderIsrUpdateLock) {
logManager.truncateTo(Map(topicPartition -> offset), isFuture = isFuture)
}
}
LogManager#truncateTo方法
基于给定的offset,截断该Partition的local log,并设置该offset为checkpoint。
/**
* Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
*
* @param partitionOffsets Partition logs that need to be truncated
* @param isFuture True iff the truncation should be performed on the future log of the specified partitions
*/
def truncateTo(partitionOffsets: Map[TopicPartition, Long], isFuture: Boolean) {
val affectedLogs = ArrayBuffer.empty[Log]
for ((topicPartition, truncateOffset) <- partitionOffsets) {
val log = {
if (isFuture)
futureLogs.get(topicPartition)
else
currentLogs.get(topicPartition)
}
// If the log does not exist, skip it
if (log != null) {
//May need to abort and pause the cleaning of the log, and resume after truncation is done.
val needToStopCleaner = cleaner != null && truncateOffset < log.activeSegment.baseOffset
if (needToStopCleaner && !isFuture)
cleaner.abortAndPauseCleaning(topicPartition)
try {
if (log.truncateTo(truncateOffset))
affectedLogs += log
if (needToStopCleaner && !isFuture)
cleaner.maybeTruncateCheckpoint(log.dir.getParentFile, topicPartition, log.activeSegment.baseOffset)
} finally {
if (needToStopCleaner && !isFuture) {
cleaner.resumeCleaning(Seq(topicPartition))
info(s"Compaction for partition $topicPartition is resumed")
}
}
}
}
for ((dir, logs) <- affectedLogs.groupBy(_.dir.getParentFile)) {
checkpointRecoveryOffsetsAndCleanSnapshot(dir, logs)
}
}
Log#truncateTo方法
基于targetOffset,对Log的activeSegment进行日志截断,使Log的最大offset小于targetOffset,并设置Log的LEO为targetOffset。删除掉LeaderEpochFileCache中startOffset大于targetOffset的所有EpochEntry。
/**
* Truncate this log so that it ends with the greatest offset < targetOffset.
*
* @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete.
* @return True iff targetOffset < logEndOffset
*/
private[log] def truncateTo(targetOffset: Long): Boolean = {
maybeHandleIOException(s"Error while truncating log to offset $targetOffset for $topicPartition in dir ${dir.getParent}") {
if (targetOffset < 0)
throw new IllegalArgumentException(s"Cannot truncate partition $topicPartition to a negative offset (%d).".format(targetOffset))
if (targetOffset >= logEndOffset) {
info(s"Truncating to $targetOffset has no effect as the largest offset in the log is ${logEndOffset - 1}")
false
} else {
//只有targetOffset小于LEO,才能进行日志截断
info(s"Truncating to offset $targetOffset")
lock synchronized {
checkIfMemoryMappedBufferClosed()
if (segments.firstEntry.getValue.baseOffset > targetOffset) {
truncateFullyAndStartAt(targetOffset)
} else {
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
deletable.foreach(deleteSegment)
//基于targetOffset,对Log的activeSegment进行日志截断
activeSegment.truncateTo(targetOffset)
//设置Log的LEO为targetOffset
updateLogEndOffset(targetOffset)
//设置Log的recoveryPoint为targetOffset(如果targetOffset更小)
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
//设置Log的logstartOffset为targetOffset(如果targetOffset更小)
this.logStartOffset = math.min(targetOffset, this.logStartOffset)
//删除掉LeaderEpochFileCache中startOffset大于targetOffset的所有EpochEntry
leaderEpochCache.foreach(_.truncateFromEnd(targetOffset))
loadProducerState(targetOffset, reloadFromCleanShutdown = false)
}
true
}
}
}
}
LogSegment#truncateTo方法
将offset索引、timestamp索引、transaction索引截断到指定的offset处。
/**
* Truncate off all index and log entries with offsets >= the given offset.
* If the given offset is larger than the largest message in this segment, do nothing.
*
* @param offset The offset to truncate to
* @return The number of log bytes truncated
*/
@nonthreadsafe
def truncateTo(offset: Long): Int = {
// Do offset translation before truncating the index to avoid needless scanning
// in case we truncate the full index
val mapping = translateOffset(offset)
//将offset索引截断到指定的offset
offsetIndex.truncateTo(offset)
//将timestamp索引截断到指定的offset
timeIndex.truncateTo(offset)
//将transaction索引截断到指定的offset
txnIndex.truncateTo(offset)
// After truncation, reset and allocate more space for the (new currently active) index
offsetIndex.resize(offsetIndex.maxIndexSize)
timeIndex.resize(timeIndex.maxIndexSize)
val bytesTruncated = if (mapping == null) 0 else log.truncateTo(mapping.position)
if (log.sizeInBytes == 0) {
created = time.milliseconds
rollingBasedTimestamp = None
}
bytesSinceLastIndexEntry = 0
if (maxTimestampSoFar >= 0)
loadLargestTimestamp()
bytesTruncated
}
ReplicaFetcherThread#fetchEpochEndOffsets方法
override def fetchEpochEndOffsets(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] = {
if (partitions.isEmpty) {
debug("Skipping leaderEpoch request since all partitions do not have an epoch")
return Map.empty
}
//创建OffsetsForLeaderEpochRequest
val epochRequest = new OffsetsForLeaderEpochRequest.Builder(offsetForLeaderEpochRequestVersion, partitions.asJava)
debug(s"Sending offset for leader epoch request $epochRequest")
try {
//向leader副本发送OffsetsForLeaderEpochRequest
val response = leaderEndpoint.sendRequest(epochRequest)
val responseBody = response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse]
debug(s"Received leaderEpoch response $response")
responseBody.responses.asScala
} catch {
case t: Throwable =>
warn(s"Error when sending leader epoch request for $partitions", t)
// if we get any unexpected exception, mark all partitions with an error
val error = Errors.forException(t)
partitions.map { case (tp, _) =>
tp -> new EpochEndOffset(error, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
}
}
}
OffsetForLeaderEpochRequest
public OffsetsForLeaderEpochRequest(Struct struct, short version) {
super(ApiKeys.OFFSET_FOR_LEADER_EPOCH, version);
epochsByPartition = new HashMap<>();
for (Object topicAndEpochsObj : struct.get(TOPICS)) {
Struct topicAndEpochs = (Struct) topicAndEpochsObj;
String topic = topicAndEpochs.get(TOPIC_NAME);
for (Object partitionAndEpochObj : topicAndEpochs.get(PARTITIONS)) {
Struct partitionAndEpoch = (Struct) partitionAndEpochObj;
int partitionId = partitionAndEpoch.get(PARTITION_ID);
int leaderEpoch = partitionAndEpoch.get(LEADER_EPOCH);
//获取当前的LeaderEpoch
Optional<Integer> currentEpoch = RequestUtils.getLeaderEpoch(partitionAndEpoch, CURRENT_LEADER_EPOCH);
TopicPartition tp = new TopicPartition(topic, partitionId);
epochsByPartition.put(tp, new PartitionData(currentEpoch, leaderEpoch));
}
}
}