kafka服务端接收生产者数据的API在
KafkaApis.scala
类中,handleProduceRequest
方法
override def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
//省略代码
request.header.apiKey match {
//生产者生产消息推送到服务端,这个接口进行处理
case ApiKeys.PRODUCE => handleProduceRequest(request, requestLocal)
//省略代码
}
}
/**
* Handle a produce request
*/
def handleProduceRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
//遍历请求中的topic,内部嵌套遍历此topic的分区,组装authorizedRequestInfo数据,当入参传入后面的appendRecords
val authorizedRequestInfo = mutable.Map[TopicPartition, MemoryRecords]()
produceRequest.data.topicData.forEach(topic => topic.partitionData.forEach { partition =>
val topicPartition = new TopicPartition(topic.name, partition.index)
// This caller assumes the type is MemoryRecords and that is true on current serialization
// We cast the type to avoid causing big change to code base.
// https://issues.apache.org/jira/browse/KAFKA-10698
val memoryRecords = partition.records.asInstanceOf[MemoryRecords]
//省略代码
ProduceRequest.validateRecords(request.header.apiVersion, memoryRecords)
authorizedRequestInfo += (topicPartition -> memoryRecords)
//省略代码
})
//省略代码
// call the replica manager to append messages to the replicas
//把消息发送到副本中
replicaManager.appendRecords(
timeout = produceRequest.timeout.toLong,
requiredAcks = produceRequest.acks,
internalTopicsAllowed = internalTopicsAllowed,
origin = AppendOrigin.CLIENT,
entriesPerPartition = authorizedRequestInfo,
requestLocal = requestLocal,
responseCallback = sendResponseCallback,
recordConversionStatsCallback = processingStatsCallback)
// if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected;
// hence we clear its data here in order to let GC reclaim its memory since it is already appended to log
produceRequest.clearPartitionRecords()
//省略代码
}
通过调用replicaManager.appendRecords
把数据存入副本中(这里的副本指的是Topic分区Leader副本)
def appendRecords(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
responseCallback: Map[TopicPartition, PartitionResponse] => Unit,
delayedProduceLock: Option[Lock] = None,
recordConversionStatsCallback: Map[TopicPartition, RecordConversionStats] => Unit = _ => (),
requestLocal: RequestLocal = RequestLocal.NoCaching): Unit = {
if (isValidRequiredAcks(requiredAcks)) {
val sTime = time.milliseconds
//追加到本地日志
val localProduceResults = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed,
origin, entriesPerPartition, requiredAcks, requestLocal)
debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
//后面逻辑都是处理响应结果localProduceResults
//将每个分区的结果转换为ProducePartitionStatus,并存储在produceStatus中。
val produceStatus = localProduceResults.map { case (topicPartition, result) =>
topicPartition -> ProducePartitionStatus(
result.info.lastOffset + 1, // required offset
new PartitionResponse(
result.error,
result.info.firstOffset.map[Long](_.messageOffset).orElse(-1L),
result.info.logAppendTime,
result.info.logStartOffset,
result.info.recordErrors,
result.info.errorMessage
)
) // response status
}
//将一些操作添加到actionQueue中,这些操作会根据result.info.leaderHwChange的值执行不同的操作。
actionQueue.add {
() =>
localProduceResults.foreach {
case (topicPartition, result) =>
val requestKey = TopicPartitionOperationKey(topicPartition)
result.info.leaderHwChange match {
case LeaderHwChange.INCREASED =>
// some delayed operations may be unblocked after HW changed
delayedProducePurgatory.checkAndComplete(requestKey)
delayedFetchPurgatory.checkAndComplete(requestKey)
delayedDeleteRecordsPurgatory.checkAndComplete(requestKey)
case LeaderHwChange.SAME =>
// probably unblock some follower fetch requests since log end offset has been updated
delayedFetchPurgatory.checkAndComplete(requestKey)
case LeaderHwChange.NONE =>
// nothing
}
}
}
//调用recordConversionStatsCallback方法,将每个分区的记录转换统计信息传递给回调函数。
recordConversionStatsCallback(localProduceResults.map { case (k, v) => k -> v.info.recordConversionStats })
//通过 delayedProduceRequestRequired 方法判断是否需要等待其它副本完成写入,如果 acks = -1,则需要等待。
if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
//根据条件判断是否需要创建延迟的produce操作。如果需要,创建一个DelayedProduce对象,并将它添加到delayedProducePurgatory中。
// create delayed produce operation
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback, delayedProduceLock)
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
//创建(主题、分区)对的列表,以用作此延迟生成操作的键
val producerRequestKeys = entriesPerPartition.keys.map(TopicPartitionOperationKey(_)).toSeq
// 再一次尝试完成该延时请求
// 如果暂时无法完成,则将对象放入到相应的Purgatory中等待后续处理
delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys)
} else {
//如果不需要延迟操作,直接将produce的结果返回给回调函数。
// we can respond immediately
val produceResponseStatus = produceStatus.map { case (k, status) => k -> status.responseStatus }
responseCallback(produceResponseStatus)
}
} else {
//每个分区创建一个错误的PartitionResponse对象,并将结果返回给回调函数。
val responseStatus = entriesPerPartition.map { case (topicPartition, _) =>
topicPartition -> new PartitionResponse(
Errors.INVALID_REQUIRED_ACKS,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.firstOffset.map[Long](_.messageOffset).orElse(-1L),
RecordBatch.NO_TIMESTAMP,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.logStartOffset
)
}
responseCallback(responseStatus)
}
}
上面写入本地日志的方法是appendToLocalLog
方法
/**
* Append the messages to the local replica logs
*/
private def appendToLocalLog(internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
requiredAcks: Short,
requestLocal: RequestLocal): Map[TopicPartition, LogAppendResult] = {
val traceEnabled = isTraceEnabled
def processFailedRecord(topicPartition: TopicPartition, t: Throwable) = {
val logStartOffset = onlinePartition(topicPartition).map(_.logStartOffset).getOrElse(-1L)
brokerTopicStats.topicStats(topicPartition.topic).failedProduceRequestRate.mark()
brokerTopicStats.allTopicsStats.failedProduceRequestRate.mark()
error(s"Error processing append operation on partition $topicPartition", t)
logStartOffset
}
//首先,它检查是否启用了跟踪(trace)日志,并根据需要记录跟踪信息。
if (traceEnabled)
trace(s"Append [$entriesPerPartition] to local log")
//遍历请求中需要把数据写入的topic集合
entriesPerPartition.map { case (topicPartition, records) =>
brokerTopicStats.topicStats(topicPartition.topic).totalProduceRequestRate.mark()
brokerTopicStats.allTopicsStats.totalProduceRequestRate.mark()
//如果topicPartition是内部主题且不允许追加记录到内部主题,则返回一个包含错误信息的LogAppendResult。
if (Topic.isInternal(topicPartition.topic) && !internalTopicsAllowed) {
(topicPartition, LogAppendResult(
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO,
Some(new InvalidTopicException(s"Cannot append to internal topic ${topicPartition.topic}"))))
} else {
try {
//尝试将records追加到相应的分区中。
val partition = getPartitionOrException(topicPartition)
val info = partition.appendRecordsToLeader(records, origin, requiredAcks, requestLocal)
val numAppendedMessages = info.numMessages
//如果追加成功,更新成功追加的字节数和消息数的统计信息。
brokerTopicStats.topicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
brokerTopicStats.allTopicsStats.bytesInRate.mark(records.sizeInBytes)
brokerTopicStats.topicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
brokerTopicStats.allTopicsStats.messagesInRate.mark(numAppendedMessages)
//省略代码
} catch {
//省略代码
}
}
}
}
appendToLocalLog
方法中比遍历Topic分区集合,针对Topic分区得到分区对象,再执行保存数据到Topic分区Leader
val partition = getPartitionOrException(topicPartition)
val info = partition.appendRecordsToLeader(records, origin, requiredAcks, requestLocal)
def appendRecordsToLeader(records: MemoryRecords, origin: AppendOrigin, requiredAcks: Int,
requestLocal: RequestLocal): LogAppendInfo = {
//函数首先获取leaderIsrUpdateLock的读锁,以确保对Leader和ISR(In-Sync Replica)的更新操作是同步的。
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
//然后检查当前是否有Leader日志,
leaderLogIfLocal match {
//如果存在Leader日志,
case Some(leaderLog) =>
//则获取最小ISR(MinInSyncReplicas)的配置和ISR的大小。
val minIsr = leaderLog.config.minInSyncReplicas
val inSyncSize = partitionState.isr.size
// Avoid writing to leader if there are not enough insync replicas to make it safe,如果没有足够的不同步副本来使其安全,请避免写入领导者
//如果ISR的大小小于最小ISR要求,并且requiredAcks的值为-1(表示不需要确认),则抛出NotEnoughReplicasException异常。
if (inSyncSize < minIsr && requiredAcks == -1) {
throw new NotEnoughReplicasException(s"The size of the current ISR ${partitionState.isr} " +
s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition")
}
//调用Leader日志的appendAsLeader方法将记录作为Leader追加到日志中,并传递相关参数。
val info = leaderLog.appendAsLeader(records, leaderEpoch = this.leaderEpoch, origin,
interBrokerProtocolVersion, requestLocal)
// we may need to increment high watermark since ISR could be down to 1,
// 我们可能需要增加高水位线,因为 ISR 可能降至 1
(info, maybeIncrementLeaderHW(leaderLog))
//如果没有,则抛出NotLeaderOrFollowerException异常。
case None =>
throw new NotLeaderOrFollowerException("Leader not local for partition %s on broker %d"
.format(topicPartition, localBrokerId))
}
}
//返回追加记录的信息,并根据是否增加了Leader高水位线,将LeaderHwChange.INCREASED或LeaderHwChange.SAME复制给返回信息的副本。
info.copy(if (leaderHWIncremented) LeaderHwChange.INCREASED else LeaderHwChange.SAME)
}
def appendAsLeader(records: MemoryRecords,
leaderEpoch: Int,
origin: AppendOrigin = AppendOrigin.CLIENT,
interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest,
requestLocal: RequestLocal = RequestLocal.NoCaching): LogAppendInfo = {
val validateAndAssignOffsets = origin != AppendOrigin.RAFT_LEADER
append(records, origin, interBrokerProtocolVersion, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), ignoreRecordSize = false)
}
private def append(records: MemoryRecords,
origin: AppendOrigin,
interBrokerProtocolVersion: MetadataVersion,
validateAndAssignOffsets: Boolean,
leaderEpoch: Int,
requestLocal: Option[RequestLocal],
ignoreRecordSize: Boolean): LogAppendInfo = {
//调用maybeFlushMetadataFile()确保在写入任何日志数据之前,分区元数据文件被写入日志目录。这样可以确保在发生故障时,可以使用正确的主题ID恢复任何日志数据。
maybeFlushMetadataFile()
//会返回一个appendInfo对象,其中包含有关记录的分析和验证结果。
val appendInfo = analyzeAndValidateRecords(records, origin, ignoreRecordSize, leaderEpoch)
//根据appendInfo.shallowCount的值判断是否有有效的消息。如果shallowCount为0,则直接返回appendInfo。
if (appendInfo.shallowCount == 0) appendInfo
else {
//对记录进行修剪以去除无效的字节或部分消息。
var validRecords = trimInvalidBytes(records, appendInfo)
//获取锁并在同步块中进行操作:
lock synchronized {
maybeHandleIOException(s"Error while appending records to $topicPartition in dir ${dir.getParent}") {
//省略代码
//可能会滚动日志,如果当前段已满。
// maybe roll the log if this segment is full
val segment = maybeRoll(validRecords.sizeInBytes, appendInfo)
//创建一个LogOffsetMetadata对象来存储日志的偏移量信息。
val logOffsetMetadata = new LogOffsetMetadata(
appendInfo.firstOrLastOffsetOfFirstBatch,
segment.baseOffset,
segment.size)
//省略代码
maybeDuplicate match {
case Some(duplicate) =>
appendInfo.setFirstOffset(Optional.of(new LogOffsetMetadata(duplicate.firstOffset)))
appendInfo.setLastOffset(duplicate.lastOffset)
appendInfo.setLogAppendTime(duplicate.timestamp)
appendInfo.setLogStartOffset(logStartOffset)
case None =>
// Before appending update the first offset metadata to include segment information
//如果没有重复的消息,则将记录追加到本地日志中,并更新高水位标记。
appendInfo.setFirstOffset(appendInfo.firstOffset.map { offsetMetadata =>
new LogOffsetMetadata(offsetMetadata.messageOffset, segment.baseOffset, segment.size)
})
//把数据追加到数据文件、索引文件、时间索引文件的方法
localLog.append(appendInfo.lastOffset, appendInfo.maxTimestamp, appendInfo.offsetOfMaxTimestamp, validRecords)
//修改高水位线最后一个日志的偏移量
updateHighWatermarkWithLogEndOffset()
//更新生产者状态
updatedProducers.values.foreach(producerAppendInfo => producerStateManager.update(producerAppendInfo))
//省略代码
//根据配置文件中刷新间隔判断,是否把所有在文件管道中的数据刷新进磁盘文件
if (localLog.unflushedMessages >= config.flushInterval) flush(false)
}
appendInfo
}
}
}
}
val segment = maybeRoll(validRecords.sizeInBytes, appendInfo)
获取需要写入到哪个segment
再调用localLog.append
执行操作
private[log] def append(lastOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = {
//给数据文件增加数据,并且根据条件判断是否给索引文件和时间索引文件增加数据
segments.activeSegment.append(largestOffset = lastOffset, largestTimestamp = largestTimestamp,
shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp, records = records)
//更新日志的结束偏移量,并更新恢复点。
updateLogEndOffset(lastOffset + 1)
}
def append(largestOffset: Long,
largestTimestamp: Long,
shallowOffsetOfMaxTimestamp: Long,
records: MemoryRecords): Unit = {
if (records.sizeInBytes > 0) {
trace(s"Inserting ${records.sizeInBytes} bytes at end offset $largestOffset at position ${log.sizeInBytes} " +
s"with largest timestamp $largestTimestamp at shallow offset $shallowOffsetOfMaxTimestamp")
//获取当前日志的物理位置
val physicalPosition = log.sizeInBytes()
//如果物理位置为0,则将rollingBasedTimestamp设置为最大时间戳。
if (physicalPosition == 0)
rollingBasedTimestamp = Some(largestTimestamp)
ensureOffsetInRange(largestOffset)
// append the messages
//追加消息到日志中,并返回追加的字节数,即追加到数据文件中
val appendedBytes = log.append(records)
trace(s"Appended $appendedBytes to ${log.file} at end offset $largestOffset")
// Update the in memory max timestamp and corresponding offset.
//更新内存中的最大时间戳和对应的偏移量
if (largestTimestamp > maxTimestampSoFar) {
maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestamp, shallowOffsetOfMaxTimestamp)
}
// append an entry to the index (if needed)
//如果距离上一次索引条目的字节数超过了indexIntervalBytes,索引文件中追加一个条目,也可能给时间索引文件增加一个条目
if (bytesSinceLastIndexEntry > indexIntervalBytes) {
offsetIndex.append(largestOffset, physicalPosition)
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar)
bytesSinceLastIndexEntry = 0
}
//更新bytesSinceLastIndexEntry的值。
bytesSinceLastIndexEntry += records.sizeInBytes
}
}
/**
* Append a set of records to the file. This method is not thread-safe and must be
* protected with a lock.
* 将一组记录追加到文件中。此方法不是线程安全的,必须使用锁进行保护。
*
* @param records The records to append
* @return the number of bytes written to the underlying file
*/
public int append(MemoryRecords records) throws IOException {
//代码检查要追加的记录的大小是否超过了当前文件位置之后的剩余空间大小,如果超过了,则抛出一个IllegalArgumentException异常。
if (records.sizeInBytes() > Integer.MAX_VALUE - size.get())
throw new IllegalArgumentException("Append of size " + records.sizeInBytes() +
" bytes is too large for segment with current file position at " + size.get());
//records.writeFullyTo(channel)方法将记录完全写入到指定的channel中,并返回实际写入的字节数。
int written = records.writeFullyTo(channel);
//ize.getAndAdd(written)方法将已写入的字节数添加到size变量中,并返回实际写入的字节数。
size.getAndAdd(written);
return written;
}
if (localLog.unflushedMessages >= config.flushInterval) flush(false)
private def flush(offset: Long, includingOffset: Boolean): Unit = {
//flushOffset 此位点之前的数据刷进磁盘中
val flushOffset = if (includingOffset) offset + 1 else offset
//offset设置为新的恢复位点,
val newRecoveryPoint = offset
val includingOffsetStr = if (includingOffset) "inclusive" else "exclusive"
maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset " +
s"($includingOffsetStr) and recovery point $newRecoveryPoint") {
//如果flushOffset大于本地文件的恢复位点
if (flushOffset > localLog.recoveryPoint) {
debug(s"Flushing log up to offset $offset ($includingOffsetStr)" +
s"with recovery point $newRecoveryPoint, last flushed: $lastFlushTime, current time: ${time.milliseconds()}," +
s"unflushed: ${localLog.unflushedMessages}")
//则把本地文件flushOffset位点之前的刷新进磁盘
localLog.flush(flushOffset)
//增加锁,同步新的恢复位点
lock synchronized {
localLog.markFlushed(newRecoveryPoint)
}
}
}
}