kafka服务端接收生产者数据的API在KafkaApis.scala类中,handleFetchRequest方法
override def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
//省略代码
request.header.apiKey match {
//消费者拉取消息请求,这个接口进行处理
case ApiKeys.FETCH => handleFetchRequest(request)
//省略代码
}
}
def handleFetchRequest(request: RequestChannel.Request): Unit = {
//从请求中获取请求的API版本(versionId)和客户端ID(clientId)。
val versionId = request.header.apiVersion
val clientId = request.header.clientId
//从请求中获取Fetch请求的数据
val fetchRequest = request.body[FetchRequest]
//根据请求的版本号,决定是否获取主题名称的映射关系(topicNames)。如果版本号大于等于13,则使用metadataCache.topicIdsToNames()获取主题名称映射关系,否则使用空的映射关系。
val topicNames =
if (fetchRequest.version() >= 13)
metadataCache.topicIdsToNames()
else
Collections.emptyMap[Uuid, String]()
//根据主题名称映射关系,获取Fetch请求的数据(fetchData)和需要忽略的主题(forgottenTopics)。
val fetchData = fetchRequest.fetchData(topicNames)
val forgottenTopics = fetchRequest.forgottenTopics(topicNames)
//创建一个Fetch上下文(fetchContext),用于管理Fetch请求的处理过程。该上下文包含了Fetch请求的版本号、元数据、是否来自Follower副本、Fetch数据、需要忽略的主题和主题名称映射关系。
val fetchContext = fetchManager.newContext(
fetchRequest.version,
fetchRequest.metadata,
fetchRequest.isFromFollower,
fetchData,
forgottenTopics,
topicNames)
//初始化两个可变数组erroneous和interesting,用于存储处理过程中的错误和请求需要哪些topic的数据。
val erroneous = mutable.ArrayBuffer[(TopicIdPartition, FetchResponseData.PartitionData)]()
val interesting = mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]()
//Fetch请求来自Follower副本
if (fetchRequest.isFromFollower) {
//则需要验证权限。如果权限验证通过
// The follower must have ClusterAction on ClusterResource in order to fetch partition data.
if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
//遍历每个分区的数据,根据不同情况将数据添加到erroneous或interesting中
fetchContext.foreachPartition { (topicIdPartition, data) =>
if (topicIdPartition.topic == null)
erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
else if (!metadataCache.contains(topicIdPartition.topicPartition))
erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
else
interesting += topicIdPartition -> data
}
} else {
//如果权限验证失败,则将所有分区的数据添加到erroneous中。
fetchContext.foreachPartition { (topicIdPartition, _) =>
erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
}
}
} else {
//如果Fetch请求来自普通的Kafka消费者
// Regular Kafka consumers need READ permission on each partition they are fetching.
val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
if (topicIdPartition.topic == null)
erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
else
partitionDatas += topicIdPartition -> partitionData
}
//需要验证对每个分区的读取权限,根据权限验证结果,将数据添加到erroneous或interesting中。
val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topicPartition.topic)
partitionDatas.foreach { case (topicIdPartition, data) =>
if (!authorizedTopics.contains(topicIdPartition.topic))
erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
else if (!metadataCache.contains(topicIdPartition.topicPartition))
erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
else
interesting += topicIdPartition -> data
}
}
//省略代码
//如果需要的topic没有校验通过或者不存在,则直接调用processResponseCallback处理响应
if (interesting.isEmpty) {
processResponseCallback(Seq.empty)
} else {
// for fetch from consumer, cap fetchMaxBytes to the maximum bytes that could be fetched without being throttled given
// no bytes were recorded in the recent quota window
// trying to fetch more bytes would result in a guaranteed throttling potentially blocking consumer progress
//如果是Follower提取数据的请求,则maxQuotaWindowBytes设置为int类型的最大,否则从记录中得到此client以前获取数据大小,
// 再和请求中、配置文件中的fetchMaxBytes比较得到下面fetchMaxBytes和fetchMinBytes两个值
val maxQuotaWindowBytes = if (fetchRequest.isFromFollower)
Int.MaxValue
else
quotas.fetch.getMaxValueInQuotaWindow(request.session, clientId).toInt
//根据请求的类型和配额限制,获取Fetch请求的最大字节数(fetchMaxBytes)和最小字节数(fetchMinBytes)
val fetchMaxBytes = Math.min(Math.min(fetchRequest.maxBytes, config.fetchMaxBytes), maxQuotaWindowBytes)
val fetchMinBytes = Math.min(fetchRequest.minBytes, fetchMaxBytes)
val clientMetadata: Optional[ClientMetadata] = if (versionId >= 11) {
// Fetch API version 11 added preferred replica logic
//提取 API 版本 11以上 添加了首选副本逻辑
Optional.of(new DefaultClientMetadata(
fetchRequest.rackId,
clientId,
request.context.clientAddress,
request.context.principal,
request.context.listenerName.value))
} else {
Optional.empty()
}
//创建一个FetchParams对象,包含了请求的各种参数
val params = new FetchParams(
versionId,
fetchRequest.replicaId,
fetchRequest.replicaEpoch,
fetchRequest.maxWait,
fetchMinBytes,
fetchMaxBytes,
FetchIsolation.of(fetchRequest),
clientMetadata
)
// call the replica manager to fetch messages from the local replica
//replicaManager.fetchMessages方法,从本地副本获取消息,并提供回调函数processResponseCallback处理响应
replicaManager.fetchMessages(
params = params,
fetchInfos = interesting,
quota = replicationQuota(fetchRequest),
responseCallback = processResponseCallback,
)
}
}
replicaManager.fetchMessages
最后通过这个方法获得日志
/**
* Fetch messages from a replica, and wait until enough data can be fetched and return;
* the callback function will be triggered either when timeout or required fetch info is satisfied.
* Consumers may fetch from any replica, but followers can only fetch from the leader.
* 从副本中获取消息,并等待可以获取足够的数据并返回;
* 当满足超时或所需的获取信息时,将触发回调函数。
* 消费者可以从任何副本中获取,但追随者只能从领导者那里获取。
*/
def fetchMessages(
params: FetchParams,
fetchInfos: Seq[(TopicIdPartition, PartitionData)],
quota: ReplicaQuota,
responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit
): Unit = {
// check if this fetch request can be satisfied right away
//调用readFromLocalLog函数从本地日志中读取消息,并将结果保存在logReadResults中。
val logReadResults = readFromLocalLog(params, fetchInfos, quota, readFromPurgatory = false)
var bytesReadable: Long = 0
var errorReadingData = false
var hasDivergingEpoch = false
var hasPreferredReadReplica = false
val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult]
//根据读取结果更新一些变量,如bytesReadable(可读取的字节数)、errorReadingData(是否读取数据时发生错误)、hasDivergingEpoch(是否存在不同的epoch)和hasPreferredReadReplica(是否存在首选读取副本)。
logReadResults.foreach { case (topicIdPartition, logReadResult) =>
brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()
brokerTopicStats.allTopicsStats.totalFetchRequestRate.mark()
if (logReadResult.error != Errors.NONE)
errorReadingData = true
if (logReadResult.divergingEpoch.nonEmpty)
hasDivergingEpoch = true
if (logReadResult.preferredReadReplica.nonEmpty)
hasPreferredReadReplica = true
bytesReadable = bytesReadable + logReadResult.info.records.sizeInBytes
logReadResultMap.put(topicIdPartition, logReadResult)
}
// respond 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 读取数据时发生错误
// 5) we found a diverging epoch 存在不同的epoch
// 6) has a preferred read replica 存在首选读取副本
if (params.maxWaitMs <= 0 || fetchInfos.isEmpty || bytesReadable >= params.minBytes || errorReadingData ||
hasDivergingEpoch || hasPreferredReadReplica) {
val fetchPartitionData = logReadResults.map { case (tp, result) =>
val isReassignmentFetch = params.isFromFollower && isAddingReplica(tp.topicPartition, params.replicaId)
tp -> result.toFetchPartitionData(isReassignmentFetch)
}
responseCallback(fetchPartitionData)
} else {
//将构建一个延迟处理的DelayedFetch对象,并将其放入延迟处理队列(delayedFetchPurgatory)中,以便在满足特定条件时完成请求。
// construct the fetch results from the read results
val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
fetchInfos.foreach { case (topicIdPartition, partitionData) =>
logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
})
}
val delayedFetch = new DelayedFetch(
params = params,
fetchPartitionStatus = fetchPartitionStatus,
replicaManager = this,
quota = quota,
responseCallback = responseCallback
)
// create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
// try to complete the request immediately, otherwise put it into the purgatory;
// this is because while the delayed fetch operation is being created, new requests
// may arrive and hence make this operation completable.
delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
}
}
通过readFromLocalLog查询数据日志
/**
* Read from multiple topic partitions at the given offset up to maxSize bytes
* 以给定的偏移量从多个主题分区读取最大最大大小字节
*/
def readFromLocalLog(
params: FetchParams,
readPartitionInfo: Seq[(TopicIdPartition, PartitionData)],
quota: ReplicaQuota,
readFromPurgatory: Boolean
): Seq[(TopicIdPartition, LogReadResult)] = {
val traceEnabled = isTraceEnabled
def read(tp: TopicIdPartition, fetchInfo: PartitionData, limitBytes: Int, minOneMessage: Boolean): LogReadResult = {
//从fetchInfo中获取一些数据,包括fetchOffset(拉取偏移量)、maxBytes(拉取的最大字节数)和logStartOffset(日志起始偏移量)。
val offset = fetchInfo.fetchOffset
val partitionFetchSize = fetchInfo.maxBytes
val followerLogStartOffset = fetchInfo.logStartOffset
//计算调整后的最大字节数adjustedMaxBytes,取fetchInfo.maxBytes和limitBytes的较小值。
val adjustedMaxBytes = math.min(fetchInfo.maxBytes, limitBytes)
try {
if (traceEnabled)
trace(s"Fetching log segment for partition $tp, offset $offset, partition fetch size $partitionFetchSize, " +
s"remaining response limit $limitBytes" +
(if (minOneMessage) s", ignoring response/partition size limits" else ""))
//获取指定分区的Partition对象
val partition = getPartitionOrException(tp.topicPartition)
//获取当前时间戳fetchTimeMs
val fetchTimeMs = time.milliseconds
//检查拉取请求或会话中的主题ID是否与日志中的主题ID一致,如果不一致则抛出InconsistentTopicIdException异常。
val topicId = if (tp.topicId == Uuid.ZERO_UUID) None else Some(tp.topicId)
if (!hasConsistentTopicId(topicId, partition.topicId))
throw new InconsistentTopicIdException("Topic ID in the fetch session did not match the topic ID in the log.")
// If we are the leader, determine the preferred read-replica
//根据一些条件选择合适的副本(replica)进行后续的数据抓取(fetch)。
val preferredReadReplica = params.clientMetadata.asScala.flatMap(
metadata => findPreferredReadReplica(partition, metadata, params.replicaId, fetchInfo.fetchOffset, fetchTimeMs))
if (preferredReadReplica.isDefined) {
//如果不存在,则跳过读取操作,直接构建一个LogReadResult对象,表示从非Leader副本获取数据的结果。
replicaSelectorOpt.foreach { selector =>
debug(s"Replica selector ${selector.getClass.getSimpleName} returned preferred replica " +
s"${preferredReadReplica.get} for ${params.clientMetadata}")
}
// If a preferred read-replica is set, skip the read
val offsetSnapshot = partition.fetchOffsetSnapshot(fetchInfo.currentLeaderEpoch, fetchOnlyFromLeader = false)
LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
divergingEpoch = None,
highWatermark = offsetSnapshot.highWatermark.messageOffset,
leaderLogStartOffset = offsetSnapshot.logStartOffset,
leaderLogEndOffset = offsetSnapshot.logEndOffset.messageOffset,
followerLogStartOffset = followerLogStartOffset,
fetchTimeMs = -1L,
lastStableOffset = Some(offsetSnapshot.lastStableOffset.messageOffset),
preferredReadReplica = preferredReadReplica,
exception = None)
} else {
// Try the read first, this tells us whether we need all of adjustedFetchSize for this partition
//尝试进行读取操作。根据读取结果构建一个LogReadResult对象,表示从分区获取数据的结果。
val readInfo: LogReadInfo = partition.fetchRecords(
fetchParams = params,
fetchPartitionData = fetchInfo,
fetchTimeMs = fetchTimeMs,
maxBytes = adjustedMaxBytes,
minOneMessage = minOneMessage,
updateFetchState = !readFromPurgatory
)
val fetchDataInfo = if (params.isFromFollower && shouldLeaderThrottle(quota, partition, params.replicaId)) {
// If the partition is being throttled, simply return an empty set.
new FetchDataInfo(readInfo.fetchedData.fetchOffsetMetadata, MemoryRecords.EMPTY)
} else if (!params.hardMaxBytesLimit && readInfo.fetchedData.firstEntryIncomplete) {
// For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
// progress in such cases and don't need to report a `RecordTooLargeException`
new FetchDataInfo(readInfo.fetchedData.fetchOffsetMetadata, MemoryRecords.EMPTY)
} else {
readInfo.fetchedData
}
//返回构建的LogReadResult对象
LogReadResult(info = fetchDataInfo,
divergingEpoch = readInfo.divergingEpoch.asScala,
highWatermark = readInfo.highWatermark,
leaderLogStartOffset = readInfo.logStartOffset,
leaderLogEndOffset = readInfo.logEndOffset,
followerLogStartOffset = followerLogStartOffset,
fetchTimeMs = fetchTimeMs,
lastStableOffset = Some(readInfo.lastStableOffset),
preferredReadReplica = preferredReadReplica,
exception = None
)
}
} catch {
//省略代码
}
}
var limitBytes = params.maxBytes
val result = new mutable.ArrayBuffer[(TopicIdPartition, LogReadResult)]
var minOneMessage = !params.hardMaxBytesLimit
readPartitionInfo.foreach { case (tp, fetchInfo) =>
val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
//记录批量的大小(以字节为单位)。
val recordBatchSize = readResult.info.records.sizeInBytes
// Once we read from a non-empty partition, we stop ignoring request and partition level size limits
//如果 recordBatchSize 大于 0,则将 minOneMessage 设置为 false,表示从非空分区读取了消息,不再忽略请求和分区级别的大小限制。
if (recordBatchSize > 0)
minOneMessage = false
limitBytes = math.max(0, limitBytes - recordBatchSize)
//将 (tp -> readResult) 添加到 result 中
result += (tp -> readResult)
}
result
}
val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
遍历主题分区分别执行read内部函数执行查询操作
方法内部通过partition.fetchRecords
查询数据
在上面readFromLocalLog
方法中,read
内部方法
val preferredReadReplica = params.clientMetadata.asScala.flatMap(
metadata => findPreferredReadReplica(partition, metadata, params.replicaId, fetchInfo.fetchOffset, fetchTimeMs))
def findPreferredReadReplica(partition: Partition,
clientMetadata: ClientMetadata,
replicaId: Int,
fetchOffset: Long,
currentTimeMs: Long): Option[Int] = {
//partition.leaderIdIfLocal返回一个Option[Int]类型的值,表示分区的领导者副本的ID。
// 如果本地是领导者副本,则返回该副本的ID,否则返回None。
partition.leaderIdIfLocal.flatMap { leaderReplicaId =>
// Don't look up preferred for follower fetches via normal replication
//如果存在领导者副本ID(leaderReplicaId),则执行flatMap中的代码块;否则直接返回None。
if (FetchRequest.isValidBrokerId(replicaId))
None
else {
replicaSelectorOpt.flatMap { replicaSelector =>
//通过metadataCache.getPartitionReplicaEndpoints方法获取分区副本的端点信息
val replicaEndpoints = metadataCache.getPartitionReplicaEndpoints(partition.topicPartition,
new ListenerName(clientMetadata.listenerName))
//创建一个可变的mutable.Set[ReplicaView]类型的集合replicaInfoSet,用于存储符合条件的副本信息。
val replicaInfoSet = mutable.Set[ReplicaView]()
//遍历分区的远程副本集合(partition.remoteReplicas),对每个副本进行以下操作:
//获取副本的状态快照(replica.stateSnapshot)。
//如果副本的brokerId存在于ISR中,并且副本的日志范围包含了指定的fetchOffset,则将副本信息添加到replicaInfoSet中。
partition.remoteReplicas.foreach { replica =>
val replicaState = replica.stateSnapshot
if (partition.inSyncReplicaIds.contains(replica.brokerId) &&
replicaState.logEndOffset >= fetchOffset &&
replicaState.logStartOffset <= fetchOffset) {
replicaInfoSet.add(new DefaultReplicaView(
replicaEndpoints.getOrElse(replica.brokerId, Node.noNode()),
replicaState.logEndOffset,
currentTimeMs - replicaState.lastCaughtUpTimeMs
))
}
}
//创建一个DefaultReplicaView对象,表示领导者副本的信息,并将其添加到replicaInfoSet中。
val leaderReplica = new DefaultReplicaView(
replicaEndpoints.getOrElse(leaderReplicaId, Node.noNode()),
partition.localLogOrException.logEndOffset,
0L
)
replicaInfoSet.add(leaderReplica)
//创建一个DefaultPartitionView对象,表示分区的信息,其中包含了副本信息集合和领导者副本信息。
val partitionInfo = new DefaultPartitionView(replicaInfoSet.asJava, leaderReplica)
//调用replicaSelector.select方法,根据特定的策略选择合适的副本。然后通过collect方法将选择的副本转换为副本的ID集合。
replicaSelector.select(partition.topicPartition, clientMetadata, partitionInfo).asScala.collect {
// Even though the replica selector can return the leader, we don't want to send it out with the
// FetchResponse, so we exclude it here
//从副本的ID集合中排除领导者副本,并返回剩余副本的ID集合。
case selected if !selected.endpoint.isEmpty && selected != leaderReplica => selected.endpoint.id
}
}
}
}
}
其中 replicaSelector.select(partition.topicPartition, clientMetadata, partitionInfo).asScala.collect
选合适副本默认首先Leader副本,但是2.4版本后支持主题分区非Leader副本中读取数据,即Follower副本读取数据
在代码上:
case selected if !selected.endpoint.isEmpty && selected != leaderReplica => selected.endpoint.id
判断设置,在配置上:
broker
端,需要配置参数 replica.selector.class
,其默认配置为LeaderSelector
,意思是:消费者从首领副本获取消息,改为RackAwareReplicaSelector
,即消费者按照指定的rack id
上的副本进行消费。还需要配置broker.rack
参数,用来指定broker
在哪个机房。consumer
端,需要配置参数client.rack
,且这个参数和broker
端的哪个broker.rack
匹配上,就会从哪个broker
上去获取消息数据。读写分离在2.4之前为什么之前不支持,后面支持了呢?
之前不支持的原因:其实对于kakfa而言,主题分区的水平扩展
完全可以解决消息的处理量,增加broker也可以降低系统负载,所以没有必要费力不讨好增加一个读写分离。
现在支持的原因:有一种场景不是很适合,跨机房或者说跨数据中心的场景,当其中一个数据中心需要向另一个数据中心同步数据的时候,如果只能从首领副本进行数据读取的话,需要跨机房来完成,而这些流量带宽又比较昂贵,而利用本地跟随者副本进行消息读取就成了比较明智的选择。
所以kafka推出这一个功能,目的并不是降低broker的系统负载,分摊消息处理量,而是为了节约流量资源。
关于Follower发请求可以看一下kafka 3.5 主题分区的Follower创建Fetcher线程从Leader拉取数据源码
def fetchRecords(
fetchParams: FetchParams,
fetchPartitionData: FetchRequest.PartitionData,
fetchTimeMs: Long,
maxBytes: Int,
minOneMessage: Boolean,
updateFetchState: Boolean
): LogReadInfo = {
def readFromLocalLog(log: UnifiedLog): LogReadInfo = {
readRecords(
log,
fetchPartitionData.lastFetchedEpoch,
fetchPartitionData.fetchOffset,
fetchPartitionData.currentLeaderEpoch,
maxBytes,
fetchParams.isolation,
minOneMessage
)
}
//判断获取数据的请求是否来自Follower
if (fetchParams.isFromFollower) {
// Check that the request is from a valid replica before doing the read
val (replica, logReadInfo) = inReadLock(leaderIsrUpdateLock) {
val localLog = localLogWithEpochOrThrow(
fetchPartitionData.currentLeaderEpoch,
fetchParams.fetchOnlyLeader
)
val replica = followerReplicaOrThrow(
fetchParams.replicaId,
fetchPartitionData
)
val logReadInfo = readFromLocalLog(localLog)
(replica, logReadInfo)
}
if (updateFetchState && !logReadInfo.divergingEpoch.isPresent) {
updateFollowerFetchState(
replica,
followerFetchOffsetMetadata = logReadInfo.fetchedData.fetchOffsetMetadata,
followerStartOffset = fetchPartitionData.logStartOffset,
followerFetchTimeMs = fetchTimeMs,
leaderEndOffset = logReadInfo.logEndOffset,
fetchParams.replicaEpoch
)
}
logReadInfo
} else {
//来自消费者客户端请求
inReadLock(`leaderIsrUpdateLock`) {
val localLog = localLogWithEpochOrThrow(
fetchPartitionData.currentLeaderEpoch,
fetchParams.fetchOnlyLeader
)
readFromLocalLog(localLog)
}
}
}
上面的方法逻辑中
//Follower的请求
val (replica, logReadInfo) = inReadLock(leaderIsrUpdateLock)
//来自消费者客户端请求
inReadLock(`leaderIsrUpdateLock`)
def readFromLocalLog(log: UnifiedLog): LogReadInfo = {
readRecords(
log,
fetchPartitionData.lastFetchedEpoch,
fetchPartitionData.fetchOffset,
fetchPartitionData.currentLeaderEpoch,
maxBytes,
fetchParams.isolation,
minOneMessage
)
}
private def readRecords(
localLog: UnifiedLog,
lastFetchedEpoch: Optional[Integer],
fetchOffset: Long,
currentLeaderEpoch: Optional[Integer],
maxBytes: Int,
fetchIsolation: FetchIsolation,
minOneMessage: Boolean
): LogReadInfo = {
//localLog的高水位标记(initialHighWatermark)、、。
val initialHighWatermark = localLog.highWatermark
//日志起始偏移(initialLogStartOffset)
val initialLogStartOffset = localLog.logStartOffset
//日志结束偏移(initialLogEndOffset)
val initialLogEndOffset = localLog.logEndOffset
//和最后一个稳定偏移(initialLastStableOffset)
val initialLastStableOffset = localLog.lastStableOffset
//省略代码
//代码调用localLog的read方法,读取指定偏移量处的数据
val fetchedData = localLog.read(
fetchOffset,
maxBytes,
fetchIsolation,
minOneMessage
)
//返回一个包含读取数据的LogReadInfo对象。
new LogReadInfo(
fetchedData,
Optional.empty(),
initialHighWatermark,
initialLogStartOffset,
initialLogEndOffset,
initialLastStableOffset
)
}
def read(startOffset: Long,
maxLength: Int,
isolation: FetchIsolation,
minOneMessage: Boolean): FetchDataInfo = {
checkLogStartOffset(startOffset)
val maxOffsetMetadata = isolation match {
case FetchIsolation.LOG_END => localLog.logEndOffsetMetadata
case FetchIsolation.HIGH_WATERMARK => fetchHighWatermarkMetadata
case FetchIsolation.TXN_COMMITTED => fetchLastStableOffsetMetadata
}
localLog.read(startOffset, maxLength, minOneMessage, maxOffsetMetadata, isolation == FetchIsolation.TXN_COMMITTED)
}
/*
*
* @param startOffset 起始偏移量(startOffset)
* @param maxLength 最大长度(maxLength)
* @param minOneMessage 是否至少读取一个消息(minOneMessage)
* @param maxOffsetMetadata 最大偏移元数据(maxOffsetMetadata)
* @param includeAbortedTxns 是否包含已中止的事务(includeAbortedTxns)
* @throws
* @return 返回一个FetchDataInfo对象
*/
def read(startOffset: Long,
maxLength: Int,
minOneMessage: Boolean,
maxOffsetMetadata: LogOffsetMetadata,
includeAbortedTxns: Boolean): FetchDataInfo = {
maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") {
trace(s"Reading maximum $maxLength bytes at offset $startOffset from log with " +
s"total length ${segments.sizeInBytes} bytes")
//获取下一个偏移元数据(endOffsetMetadata)和对应的偏移量(endOffset)
val endOffsetMetadata = nextOffsetMetadata
val endOffset = endOffsetMetadata.messageOffset
//获得segment的集合,比如会获得某个位点后所有的segment的列表,有序
var segmentOpt = segments.floorSegment(startOffset)
// return error on attempt to read beyond the log end offset
//如果起始偏移量大于结束偏移量或者找不到日志段,则抛出OffsetOutOfRangeException异常。
if (startOffset > endOffset || segmentOpt.isEmpty)
throw new OffsetOutOfRangeException(s"Received request for offset $startOffset for partition $topicPartition, " +
s"but we only have log segments upto $endOffset.")
//如果起始偏移量等于最大偏移量元数据的偏移量,函数返回一个空的FetchDataInfo对象
if (startOffset == maxOffsetMetadata.messageOffset)
emptyFetchDataInfo(maxOffsetMetadata, includeAbortedTxns)
else if (startOffset > maxOffsetMetadata.messageOffset)
//如果起始偏移量大于最大偏移量元数据的偏移量,函数返回一个空的FetchDataInfo对象,并将起始偏移量转换为偏移元数据
emptyFetchDataInfo(convertToOffsetMetadataOrThrow(startOffset), includeAbortedTxns)
else {
//函数在小于目标偏移量的基本偏移量的日志段上进行读取
var fetchDataInfo: FetchDataInfo = null
//首先fetchDataInfo不为null,和大于start位点的segment要存在
while (fetchDataInfo == null && segmentOpt.isDefined) {
val segment = segmentOpt.get
val baseOffset = segment.baseOffset
val maxPosition =
// Use the max offset position if it is on this segment; otherwise, the segment size is the limit.
//如果它在此段上,请使用最大偏移位置;否则,段大小是限制。
if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment
else segment.size
fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage)
if (fetchDataInfo != null) {
//则根据条件判断,如果includeAbortedTxns为真,则调用addAbortedTransactions方法添加中断的事务到fetchDataInfo中。
if (includeAbortedTxns)
fetchDataInfo = addAbortedTransactions(startOffset, segment, fetchDataInfo)
}
//如果fetchDataInfo为null,则将segmentOpt设置为segments中大于baseOffset的下一个段。
else segmentOpt = segments.higherSegment(baseOffset)
}
//成功读取到消息,函数返回FetchDataInfo对象
if (fetchDataInfo != null) fetchDataInfo
else {
//如果已经超过了最后一个日志段的末尾且没有读取到任何数据,则返回一个空的FetchDataInfo对象,其中包含下一个偏移元数据和空的内存记录(MemoryRecords.EMPTY)
new FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
}
}
}
}
首先获得segment列表var segmentOpt = segments.floorSegment(startOffset)
,
通过 fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage)
从segment获取数据
def read(startOffset: Long,
maxSize: Int,
maxPosition: Long = size,
minOneMessage: Boolean = false): FetchDataInfo = {
if (maxSize < 0)
throw new IllegalArgumentException(s"Invalid max size $maxSize for log read from segment $log")
val startOffsetAndSize = translateOffset(startOffset)
// if the start position is already off the end of the log, return null
//则表示起始位置已经超出了日志的末尾,则返回 null
if (startOffsetAndSize == null)
return null
//起始偏移量、基准偏移量和起始位置创建一个LogOffsetMetadata对象
val startPosition = startOffsetAndSize.position
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
val adjustedMaxSize =
if (minOneMessage) math.max(maxSize, startOffsetAndSize.size)
else maxSize
// return a log segment but with zero size in the case below
if (adjustedMaxSize == 0)
return new FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY)
// calculate the length of the message set to read based on whether or not they gave us a maxOffset
//根据给定的maxOffset计算要读取的消息集的长度,将其限制为maxPosition和起始位置之间的较小值,并将结果赋给fetchSize变量。
val fetchSize: Int = min((maxPosition - startPosition).toInt, adjustedMaxSize)
//创建一个FetchDataInfo对象,其中包含偏移量元数据、从起始位置开始的指定大小的日志切片(log slice)以及其他相关信息
//其中log.slice(startPosition, fetchSize)是日志数据
new FetchDataInfo(offsetMetadata, log.slice(startPosition, fetchSize),
adjustedMaxSize < startOffsetAndSize.size, Optional.empty())
}
log.slice
获取文件数据
public FileRecords slice(int position, int size) throws IOException {
int availableBytes = availableBytes(position, size);
int startPosition = this.start + position;
return new FileRecords(file, channel, startPosition, startPosition + availableBytes, true);
}
这里生成一个新的文件数据对象,下面就是FileRecords
的构造方法
FileRecords(File file,
FileChannel channel,
int start,
int end,
boolean isSlice) throws IOException {
this.file = file;
this.channel = channel;
this.start = start;
this.end = end;
this.isSlice = isSlice;
this.size = new AtomicInteger();
//表示这只是一个切片视图,不需要检查文件大小,直接将size设置为end - start。
if (isSlice) {
// don't check the file size if this is just a slice view
size.set(end - start);
} else {
//如果isSlice为false,表示这不是一个切片,需要检查文件的大小。如果文件大小超过了Integer.MAX_VALUE,将抛出KafkaException异常。
if (channel.size() > Integer.MAX_VALUE)
throw new KafkaException("The size of segment " + file + " (" + channel.size() +
") is larger than the maximum allowed segment size of " + Integer.MAX_VALUE);
//否则,将文件大小和end之间的较小值设置为limit,并将size设置为limit - start。然后,将文件通道的位置设置为limit,即文件末尾的位置。
int limit = Math.min((int) channel.size(), end);
size.set(limit - start);
// if this is not a slice, update the file pointer to the end of the file
// set the file position to the last byte in the file
channel.position(limit);
}
batches = batchesFrom(start);
}
/**
* Get an iterator over the record batches in the file, starting at a specific position. This is similar to
* {@link #batches()} except that callers specify a particular position to start reading the batches from. This
* method must be used with caution: the start position passed in must be a known start of a batch.
* @param start The position to start record iteration from; must be a known position for start of a batch
* @return An iterator over batches starting from {@code start}
*/
//它的作用是从FileRecords直接返回一个batch的iterator
public Iterable<FileChannelRecordBatch> batchesFrom(final int start) {
return () -> batchIterator(start);
}
private AbstractIterator<FileChannelRecordBatch> batchIterator(int start) {
final int end;
if (isSlice)
end = this.end;
else
end = this.sizeInBytes();
//创建一个FileLogInputStream对象inputStream,并传入this、start和end作为参数。
FileLogInputStream inputStream = new FileLogInputStream(this, start, end);
//创建一个RecordBatchIterator对象,并将inputStream作为参数传入。
//将创建的RecordBatchIterator对象作为返回值返回。
return new RecordBatchIterator<>(inputStream);
}
}
FileLogInputStream
类实现了nextBatch()
接口,这个接口是从基础输入流中获取下一个记录批次。
public class FileLogInputStream implements LogInputStream<FileLogInputStream.FileChannelRecordBatch> {
/**
* Create a new log input stream over the FileChannel
* @param records Underlying FileRecords instance
* @param start Position in the file channel to start from
* @param end Position in the file channel not to read past
*/
FileLogInputStream(FileRecords records,
int start,
int end) {
this.fileRecords = records;
this.position = start;
this.end = end;
}
@Override
public FileChannelRecordBatch nextBatch() throws IOException {
//首先获取文件的通道(channel)
FileChannel channel = fileRecords.channel();
//检查是否达到了文件末尾或者下一个记录批次的起始位置。如果达到了文件末尾,则返回空(null)。
if (position >= end - HEADER_SIZE_UP_TO_MAGIC)
return null;
//读取文件通道中的记录头部数据,并将其存储在一个缓冲区(logHeaderBuffer)
logHeaderBuffer.rewind();
Utils.readFullyOrFail(channel, logHeaderBuffer, position, "log header");
//记录头部数据中解析出偏移量(offset)和记录大小(size)
logHeaderBuffer.rewind();
long offset = logHeaderBuffer.getLong(OFFSET_OFFSET);
int size = logHeaderBuffer.getInt(SIZE_OFFSET);
// V0 has the smallest overhead, stricter checking is done later
if (size < LegacyRecord.RECORD_OVERHEAD_V0)
throw new CorruptRecordException(String.format("Found record size %d smaller than minimum record " +
"overhead (%d) in file %s.", size, LegacyRecord.RECORD_OVERHEAD_V0, fileRecords.file()));
//检查是否已经超过了文件末尾减去记录开销和记录大小的位置。如果超过了,则返回空(null)
if (position > end - LOG_OVERHEAD - size)
return null;
//代码会根据记录头部的(magic)
byte magic = logHeaderBuffer.get(MAGIC_OFFSET);
//创建一个记录批次对象(batch)
final FileChannelRecordBatch batch;
if (magic < RecordBatch.MAGIC_V个LUE_V2)
//则创建一个旧版本的记录批次对象
batch = new LegacyFileChannelRecordBatch(offset, magic, fileRecords, position, size);
else
//否则创建一个默认版本的记录批次对象
batch = new DefaultFileChannelRecordBatch(offset, magic, fileRecords, position, size);
//代码会更新当前位置(position),以便下次读取下一个记录批次。
position += batch.sizeInBytes();
return batch;
}
}
上文中的batchIterator
方法会把文件流构造RecordBatchIterator
对象
class RecordBatchIterator<T extends RecordBatch> extends AbstractIterator<T> {
private final LogInputStream<T> logInputStream;
RecordBatchIterator(LogInputStream<T> logInputStream) {
this.logInputStream = logInputStream;
}
@Override
protected T makeNext() {
try {
T batch = logInputStream.nextBatch();
if (batch == null)
return allDone();
return batch;
} catch (EOFException e) {
throw new CorruptRecordException("Unexpected EOF while attempting to read the next batch", e);
} catch (IOException e) {
throw new KafkaException(e);
}
}
}
AbstractIterator
抽象类
public abstract class AbstractIterator<T> implements Iterator<T> {
private enum State {
READY, NOT_READY, DONE, FAILED
}
private State state = State.NOT_READY;
private T next;
@Override
public boolean hasNext() {
switch (state) {
case FAILED:
throw new IllegalStateException("Iterator is in failed state");
case DONE:
return false;
case READY:
return true;
default:
return maybeComputeNext();
}
}
@Override
public T next() {
if (!hasNext())
throw new NoSuchElementException();
state = State.NOT_READY;
if (next == null)
throw new IllegalStateException("Expected item but none found.");
return next;
}
@Override
public void remove() {
throw new UnsupportedOperationException("Removal not supported");
}
public T peek() {
if (!hasNext())
throw new NoSuchElementException();
return next;
}
protected T allDone() {
state = State.DONE;
return null;
}
protected abstract T makeNext();
private Boolean maybeComputeNext() {
state = State.FAILED;
next = makeNext();
if (state == State.DONE) {
return false;
} else {
state = State.READY;
return true;
}
}
}
调用RecordBatchIterator
类的makeNext
()方法,之后调用第五章节的FileLogInputStream
中的nextBatch()
DefaultFileChannelRecordBatch
这个是默认的
static class DefaultFileChannelRecordBatch extends FileLogInputStream.FileChannelRecordBatch {
DefaultFileChannelRecordBatch(long offset,
byte magic,
FileRecords fileRecords,
int position,
int batchSize) {
super(offset, magic, fileRecords, position, batchSize);
}
@Override
protected RecordBatch toMemoryRecordBatch(ByteBuffer buffer) {
return new DefaultRecordBatch(buffer);
}
@Override
public long baseOffset() {
return offset;
}
//省略代码
}
之后看一下哪里调用的
DefaultFileChannelRecordBatch
中的toMemoryRecordBatch
方法
DefaultRecordBatch
,再通过这个batch
的iterator
方法获取到Iterator
的
public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch {
@Override
public Iterator<Record> iterator() {
if (count() == 0)
return Collections.emptyIterator();
if (!isCompressed())
return uncompressedIterator();
// for a normal iterator, we cannot ensure that the underlying compression stream is closed,
// so we decompress the full record set here. Use cases which call for a lower memory footprint
// can use `streamingIterator` at the cost of additional complexity
try (CloseableIterator<Record> iterator = compressedIterator(BufferSupplier.NO_CACHING, false)) {
List<Record> records = new ArrayList<>(count());
while (iterator.hasNext())
records.add(iterator.next());
return records.iterator();
}
}
}
DefaultFileChannelRecordBatch
是FileChannelRecordBatch
的一个子类。FileChannelRecordBatch
表示日志是通过FileChannel
的形式来保存的。在遍历日志的时候不需要将日志全部读到内存中,而是在需要的时候再读取。我们直接看最重要的iterator方法
public abstract static class FileChannelRecordBatch extends AbstractRecordBatch {
protected final long offset;
protected final byte magic;
protected final FileRecords fileRecords;
protected final int position;
protected final int batchSize;
private RecordBatch fullBatch;
private RecordBatch batchHeader;
FileChannelRecordBatch(long offset,
byte magic,
FileRecords fileRecords,
int position,
int batchSize) {
this.offset = offset;
this.magic = magic;
this.fileRecords = fileRecords;
this.position = position;
this.batchSize = batchSize;
}
//省略代码
@Override
public Iterator<Record> iterator() {
return loadFullBatch().iterator();
}
//省略代码
}
protected RecordBatch loadFullBatch() {
if (fullBatch == null) {
batchHeader = null;
fullBatch = loadBatchWithSize(sizeInBytes(), "full record batch");
}
return fullBatch;
}
最后会调用DefaultFileChannelRecordBatch
类型的toMemoryRecordBatch
方法在内存中生成批量数据
private RecordBatch loadBatchWithSize(int size, String description) {
FileChannel channel = fileRecords.channel();
try {
ByteBuffer buffer = ByteBuffer.allocate(size);
Utils.readFullyOrFail(channel, buffer, position, description);
buffer.rewind();
//在内存中生成数据
return toMemoryRecordBatch(buffer);
} catch (IOException e) {
throw new KafkaException("Failed to load record batch at position " + position + " from " + fileRecords, e);
}
}