文章目錄
系列前言
kafka作爲一個處理實時數據和日誌的管道,每秒可以處理幾十萬條消息。其瓶頸自然也在I/O層面,所以其高吞吐背後離不開如下幾個特性:
- NIO
- 磁盤順序讀寫
- Queue數據結構的極致使用
- 分區提高併發
- 零拷貝提高效率
- 異步刷盤
- 壓縮提高數據傳輸效率
本次我將從kafka-2.2.0的源碼分析其順序寫入與刷盤機制的細節。
一、日誌寫入
1.Log文件結構簡介
kafka的日誌文件
kafka-logs/${topic}-${partition}/:
|-- 00000000000000000000.index
|-- 00000000000000000000.log
|-- 00000000000000000000.timeindex
|-- 00000000000000002309.snapshot
|-- 00000000000000167080.snapshot
`-- leader-epoch-checkpoint
A segment of the log 由3部分組成:
- log: FileRecord,即實際的消息
- index: 索引
- timeindex: 時間索引
其中:
- 命名規則爲 segment 文件最後一條消息的 offset 值。
- log.segment.bytes 日誌切割(默認 1G)
這裏不詳細介紹Log文件結構以及message從接收到處理的過程.
(補充問題:在partition中如何通過offset查找message)
2.寫入過程
// org.apache.kafka.common.record.FileRecords
/**
* 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 {
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());
int written = records.writeFullyTo(channel);
size.getAndAdd(written);
return written;
}
/**
* Write all records to the given channel (including partial records).
* @param channel The channel to write to
* @return The number of bytes written
* @throws IOException For any IO errors writing to the channel
*/
public int writeFullyTo(GatheringByteChannel channel) throws IOException {
buffer.mark();
int written = 0;
while (written < sizeInBytes())
written += channel.write(buffer);
buffer.reset();
return written;
}
-
每個分區寫入過程沒有帶offset,這種append-only的寫法保證了順序寫入,一定程度降低磁盤負載(避免隨機寫操帶來的頻繁磁盤尋道問題)。
-
由於kafka的網絡模型是1+N+M,也就意味着M個worker線程可能寫同1個log文件,append顯然不是線程安全的,上層調用時需要加鎖。
-
此時僅僅寫入文件系統的PageCache(內存)中,
- 不做特殊操作的話,將由操作系統決定什麼時候把 OS Cache 裏的數據真的刷入磁盤文件中。
- kafka本身提供強制刷盤機制來強制刷盤,下文將詳細介紹
附加鎖寫入:
Log -> LogSegment -> FileRecords
// kafka.log.Log
private def append() ={
……
lock synchronized {
……
segment.append(largestOffset = appendInfo.lastOffset,
largestTimestamp = appendInfo.maxTimestamp,
shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp,
records = validRecords)
……
}
}
二、刷盤分析
1.刷盤參數
kafka提供3個參數來優化刷盤機制
- log.flush.interval.messages //多少條消息刷盤1次
- log.flush.interval.ms //隔多長時間刷盤1次
- log.flush.scheduler.interval.ms //週期性的刷盤。
kafka配置獲取入口
// kafka.server.KafkaConfig
val LogFlushIntervalMessagesProp = "log.flush.interval.messages"
val LogFlushSchedulerIntervalMsProp = "log.flush.scheduler.interval.ms"
val LogFlushIntervalMsProp = "log.flush.interval.ms"
2.參數詳解與刷盤源碼解讀
2.1 log.flush.interval.messages參數
log.flush.interval.messages
即多少條消息刷盤1次,這個參數在Log
類中使用。
Log
類是append-only的LogSegments的序列。在append的時候直接判斷未刷新的message數是否達到閾值log.flush.interval.messages
// kafka.log.Log
class Log(……){
……
private def append(……): LogAppendInfo = {
……
if (unflushedMessages >= config.flushInterval)
flush()
appendInfo
}
……
}
2.2 log.flush.interval.ms與log.flush.scheduler.interval.ms參數
log.flush.interval.ms
與log.flush.scheduler.interval.ms
2個參數需要相互配合,在LogManager
使用。
LogManager
類是kafka日誌管理子系統的入口,負責日誌的創建、檢索和清理。LogManager
啓動一個調度線程,根據flushCheckMs
週期執行flushDirtyLogs
方法
注: flushCheckMs
即週期性刷盤參數log.flush.scheduler.interval.ms
。
// kafka.log.LogManager
def startup() {
/* Schedule the cleanup task to delete old logs */
if (scheduler != null) {
info("Starting log cleanup with a period of %d ms.".format(retentionCheckMs))
scheduler.schedule("kafka-log-retention",
cleanupLogs _,
delay = InitialTaskDelayMs,
period = retentionCheckMs,
TimeUnit.MILLISECONDS)
info("Starting log flusher with a default period of %d ms.".format(flushCheckMs))
scheduler.schedule("kafka-log-flusher",
flushDirtyLogs _,
delay = InitialTaskDelayMs,
period = flushCheckMs,
TimeUnit.MILLISECONDS)
scheduler.schedule("kafka-recovery-point-checkpoint",
checkpointLogRecoveryOffsets _,
delay = InitialTaskDelayMs,
period = flushRecoveryOffsetCheckpointMs,
TimeUnit.MILLISECONDS)
scheduler.schedule("kafka-log-start-offset-checkpoint",
checkpointLogStartOffsets _,
delay = InitialTaskDelayMs,
period = flushStartOffsetCheckpointMs,
TimeUnit.MILLISECONDS)
scheduler.schedule("kafka-delete-logs", // will be rescheduled after each delete logs with a dynamic period
deleteLogs _,
delay = InitialTaskDelayMs,
unit = TimeUnit.MILLISECONDS)
}
if (cleanerConfig.enableCleaner)
cleaner.startup()
}
flushDirtyLogs調用時會計算上次flush的間隔,對比log.flush.interval.ms
配置對應的flushMs
變量決定是否flush
// kafka.log.LogManager
private def flushDirtyLogs(): Unit = {
debug("Checking for dirty logs to flush...")
for ((topicPartition, log) <- currentLogs.toList ++ futureLogs.toList) {
try {
// 求出上次flush的間隔
val timeSinceLastFlush = time.milliseconds - log.lastFlushTime
debug(s"Checking if flush is needed on ${topicPartition.topic} flush interval ${log.config.flushMs}" +
s" last flushed ${log.lastFlushTime} time since last flush: $timeSinceLastFlush")
if(timeSinceLastFlush >= log.config.flushMs)
log.flush
} catch {
case e: Throwable =>
error(s"Error flushing topic ${topicPartition.topic}", e)
}
}
}
最終實現使用用FileRecords
的flush
方法,即調用JDK的java.nio.channels.FileChannel
的force
方法強制刷盤
// org.apache.kafka.common.record.FileRecords
/**
* Commit all written data to the physical disk
*/
public void flush() throws IOException {
channel.force(true);
}
最後
實際上,官方不建議通過上述的刷盤3個參數來強制寫盤。其認爲數據的可靠性通過replica來保證,而強制flush數據到磁盤會對整體性能產生影響。