日志压缩
在很多实践的场景中,key和value的值是不丹变化的,如果消费者只关心key对应的最新value值,可以开启kafka的日志压缩功能,服务端要在后台启动Cleaner线程池,把相同key的消息进行合并。
每个Log都可以通过clean checkpoint部分切分成clean和dirty两部分,clean表示之前已经被压缩过的消息,dirty表示没有被压缩过的消息。
经过压缩之后,clean部分的offset可能是断断续续的,而dirty部门都是连续递增的。
每个Log需要进行日志压缩的迫切程度不同,每个Cleaner只选取最需要被压缩的Log进行处理,迫切程度是通过cleanableRatio决定的。
1. Cleander线程在选定需要清理的Log后,首先为dirty部分建立key和last_offset(这个key出现的最大offset)的对应关系。
2. 重新复制LogSegment,只保留last_offset中记录的消息,抛弃其他消息。
3. 对相邻的LogSegment进行合并,避免出现过小的日志文件和索引文件。
4. value为空的消息会被认为是删除此key对应的消息的标志,此标志消息会被保留一段时间。
LogCleanManager中各个字段如下:
checkpoints:Map[File, OffsetCheckpoint]类型,用来维护data数据目录与cleaner-offset-checkpoint文件之间的对应关系。
inProgress:HashMap[TopicAndPartition, LogCleaningState]: 用于记录正在进行清理的TopicAndPartition的压缩状态。
lock:保护checkpoints集合和inProgress集合锁
pausedCleaningCond:线程阻塞等待压缩状态由LogCleaningAborted转换为LogCleaningPaused
1. 当开始进行压缩的时候,会先进入LogCleanInProgress状态
2. 压缩任务被暂停时进入LogCleaningPaused
3. 压缩任务被中断,则进入LogCleaningPaused状态。
4. 处于LogCleaningPaused状态的topicAndPartition的日志不会被压缩,知道有其他线程恢复它的状态。
def grabFilthiestLog(): Option[LogToClean] = {
inLock(lock) { //加锁
//拿到全部log的cleanercheckpoint
val lastClean = allCleanerCheckpoints()
val dirtyLogs = logs.filter {
//过滤配置为delete的Log
case (topicAndPartition, log) => log.config.compact // skip any logs marked for delete rather than dedupe
}.filterNot {
//过滤掉包含inProgress状态的log
case (topicAndPartition, log) => inProgress.contains(topicAndPartition) // skip any logs already in-progress
}.map {
case (topicAndPartition, log) => // create a LogToClean instance for each
// if the log segments are abnormally truncated and hence the checkpointed offset
// is no longer valid, reset to the log starting offset and log the error event
//获取Log的第一条消息的offset
val logStartOffset = log.logSegments.head.baseOffset
//决定最终压缩开始的位置,firstDirtyOffset的值可能是logStartOffset,也可能是clean checkpoint
val firstDirtyOffset = {
val offset = lastClean.getOrElse(topicAndPartition, logStartOffset)
if (offset < logStartOffset) {
error("Resetting first dirty offset to log start offset %d since the checkpointed offset %d is invalid."
.format(logStartOffset, offset))
logStartOffset
} else {
offset
}
}
为Log创建一个LogToClean对象,在LogToClean对象用维护每个Log的clean部分字节数、dirty部分字节数,以及cleanableRatio
LogToClean(topicAndPartition, log, firstDirtyOffset)
}.filter(ltc => ltc.totalBytes > 0) // skip any empty logs
//获取dirtyLogs集合中cleanableRatio的最大值。
this.dirtiestLogCleanableRatio = if (!dirtyLogs.isEmpty) dirtyLogs.max.cleanableRatio else 0
// 过滤掉cleanableRatio小于配置的minCleanableRatio值的Log
val cleanableLogs = dirtyLogs.filter(ltc => ltc.cleanableRatio > ltc.log.config.minCleanableRatio)
if(cleanableLogs.isEmpty) {
None
} else {
val filthiest = cleanableLogs.max //选择要压缩的Log
//更新此分区对应的压缩状态到inProgress
inProgress.put(filthiest.topicPartition, LogCleaningInProgress)
Some(filthiest)
}
}
}
CleanThread是执行日志压缩的操作线程:
val cleaner = new Cleaner(id = threadId,
//dirty部分key和last_offset的关系
offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt,
hashAlgorithm = config.hashAlgorithm),
// 指定读写LogSegment的ByteBuffer大小
ioBufferSize = config.ioBufferSize / config.numThreads / 2,
//消息的最大长度
maxIoBufferSize = config.maxMessageSize,
//SkimpyOffsetMap的最大占用比例
dupBufferLoadFactor = config.dedupeBufferLoadFactor,
throttler = throttler,
time = time,
//压缩状态
checkDone = checkDone)
CleanThread.doWork实现的压缩逻辑:
override def doWork() {
cleanOrSleep()
}
private def cleanOrSleep() {
//通过grabFilthiestLog获取需要进行压缩的Log
cleanerManager.grabFilthiestLog() match {
case None =>
// 没有需要压缩的Log,就睡眠一段时间
backOffWaitLatch.await(config.backOffMs, TimeUnit.MILLISECONDS)
case Some(cleanable) =>
// there's a log, clean it
var endOffset = cleanable.firstDirtyOffset
try {
//clean对日志进行压缩
endOffset = cleaner.clean(cleanable)
recordStats(cleaner.id, cleanable.log.name, cleanable.firstDirtyOffset, endOffset, cleaner.stats)
} catch {
case pe: LogCleaningAbortedException => // task can be aborted, let it go.
} finally {
//对Log的压缩状态进行转化,同时更新cleaner-offset-checkpoint文件
cleanerManager.doneCleaning(cleanable.topicPartition, cleanable.log.dir.getParentFile, endOffset)
}
}
}
private[log] def clean(cleanable: LogToClean): Long = {
stats.clear()
info("Beginning cleaning of log %s.".format(cleanable.log.name))
val log = cleanable.log
// build the offset map
info("Building offset map for %s...".format(cleanable.log.name))
//确定压缩的最大offset上限,因为activeSegment不参与压缩,所以它的baseOffset就是要压缩的消息的最大offset
val upperBoundOffset = log.activeSegment.baseOffset
//遍历LogSegment,填充OffsetMap,记录每个key值应该要保留的offset,当Offset
val endOffset = buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap) + 1
stats.indexDone()
// 计算可以安全删除value为空的消息的LogSegment。
val deleteHorizonMs =
log.logSegments(0, cleanable.firstDirtyOffset).lastOption match {
case None => 0L
case Some(seg) => seg.lastModified - log.config.deleteRetentionMs
}
// 对压缩的消息进行分组,并分组进行clean
info("Cleaning log %s (discarding tombstones prior to %s)...".format(log.name, new Date(deleteHorizonMs)))
for (group <- groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, log.config.maxIndexSize))
cleanSegments(log, group, offsetMap, deleteHorizonMs)
// record buffer utilization
stats.bufferUtilization = offsetMap.utilization
stats.allDone()
endOffset
}
buildOffsetMap方法主要从firstDirtyOffset开始遍历LogSegment,并填充OffsetMap。在填充的过程中,之后加入到Log中的消息会覆盖之前相同offfset的消息。
private[log] def buildOffsetMap(log: Log, start: Long, end: Long, map: OffsetMap): Long = {
map.clear()
//查找从start-end所有的LogSegment
val dirty = log.logSegments(start, end).toBuffer
info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, dirty.size, start, end))
// Add all the dirty segments. We must take at least map.slots * load_factor,
// but we may be able to fit more (if there is lots of duplication in the dirty section of the log)
var offset = dirty.head.baseOffset
require(offset == start, "Last clean offset is %d but segment base offset is %d for log %s.".format(start, offset, log.name))
//是否填充满了
var full = false
//遍历dirty集合,循环条件是OffsetMap未填充满
for (segment <- dirty if !full) {
//检查分区的压缩状态
checkDone(log.topicAndPartition)
//处理单个LogSegment,把消息的key和offset添加到OffsetMap中
val newOffset = buildOffsetMapForSegment(log.topicAndPartition, segment, map)
if (newOffset > -1L) //OffsetMap未满,移动offset
offset = newOffset
else {
// If not even one segment can fit in the map, compaction cannot happen
// 如果offset>start说明OffsetMap中记录了最后一个LogSegment的一部分消息
require(offset > start, "Unable to build the offset map for segment %s/%s. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads".format(log.name, segment.log.file.getName))
debug("Offset map is full, %d segments fully mapped, segment with base offset %d is partially mapped".format(dirty.indexOf(segment), segment.baseOffset))
full = true
}
}
info("Offset map for log %s complete.".format(log.name))
offset
}
buildOffsetMapForSegment完成对OffsetMap的填充
private def buildOffsetMapForSegment(topicAndPartition: TopicAndPartition, segment: LogSegment, map: OffsetMap): Long = {
var position = 0
var offset = segment.baseOffset
val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt
while (position < segment.log.sizeInBytes) { //遍历LogSegment
checkDone(topicAndPartition)//检查压缩状态
readBuffer.clear()
//从LogSegment中读取消息
val messages = new ByteBufferMessageSet(segment.log.readInto(readBuffer, position))
throttler.maybeThrottle(messages.sizeInBytes)
val startPosition = position
//遍历ByteBufferMessage(深层迭代)
for (entry <- messages) {
val message = entry.message
if (message.hasKey) {//只处理有key的消息
if (map.size < maxDesiredMapSize)
//把key和offset放在OffsetMap中
map.put(message.key, entry.offset)
else {
// The map is full, stop looping and return
return -1L
}
}
offset = entry.offset
stats.indexMessagesRead(1)
}
//移动position准备下次读取
position += messages.validBytes
stats.indexBytesRead(messages.validBytes)
// position没有移动,表示没有读到一个完整Message,对readBuffer和writeBuffer进行扩容,再重新读取
if(position == startPosition)
growBuffers()
}
//重置readBuffer和writeBuffer的大小
restoreBuffers()
//返回此LogSegment的最后一个消息的offset
offset
}
在这次日志压缩的结束位置endOffset后,通过groupSegmentsBySize对0到endOffset进行分组,分组的单位是LogSegment,即每个LogSegment只属于一组。返回分组后的LogSegment集合会传人cleanSegment中,
private[log] def cleanSegments(log: Log,
segments: Seq[LogSegment],
map: OffsetMap,
deleteHorizonMs: Long) {
// 创建.clean文件和索引文件,文件名是分组的第一个LogSegment的baseOffset
val logFile = new File(segments.head.log.file.getPath + Log.CleanedFileSuffix)
logFile.delete()
val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix)
indexFile.delete()
//刷新对应的FileMessageSet对象,OffsetIndex对象和LogSegment对象
val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate)
val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
try {
// 判定LogSegment中的删除标志是否可以安全删除
for (old <- segments) {
val retainDeletes = old.lastModified > deleteHorizonMs
info("Cleaning segment %s in log %s (last modified %s) into %s, %s deletes."
.format(old.baseOffset, log.name, new Date(old.lastModified), cleaned.baseOffset, if(retainDeletes) "retaining" else "discarding"))
cleanInto(log.topicAndPartition, old, cleaned, map, retainDeletes, log.config.messageFormatVersion.messageFormatVersion)
}
// 截断多余的索引项
index.trimToValidSize()
// 刷新磁盘
cleaned.flush()
// 更新修改时间
val modified = segments.last.lastModified
cleaned.lastModified = modified
// 文件从.clean改成.swap,把cleaned对象加入到跳表中,把分组中的LogSegment从segments中删除,把.swap文件删除。
info("Swapping in cleaned segment %d for segment(s) %s in log %s.".format(cleaned.baseOffset, segments.map(_.baseOffset).mkString(","), log.name))
log.replaceSegments(cleaned, segments)
} catch {
case e: LogCleaningAbortedException =>
cleaned.delete()
throw e
}
}
cleanInto()方法对一个LogSegment分组进行压缩
private[log] def cleanInto(topicAndPartition: TopicAndPartition,
source: LogSegment,
dest: LogSegment,
map: OffsetMap,
retainDeletes: Boolean,
messageFormatVersion: Byte) {
var position = 0
while (position < source.log.sizeInBytes) {//遍历待压缩的LogSegment
//检测压缩状态
checkDone(topicAndPartition)
// read a chunk of messages and copy any that are to be retained to the write buffer to be written out
readBuffer.clear()
writeBuffer.clear()
//读取消息
val messages = new ByteBufferMessageSet(source.log.readInto(readBuffer, position))
throttler.maybeThrottle(messages.sizeInBytes)
// 遍历每个消息查看他是否保留
var messagesRead = 0
for (entry <- messages.shallowIterator) {
val size = MessageSet.entrySize(entry.message)
stats.readMessage(size)
if (entry.message.compressionCodec == NoCompressionCodec) {
//判断是否保留有三个条件:
//1. 消息是否有key
//2. .OffsetMap是否有相同key且offset更大的消息。
//3. 此消息是删除标志,且此LogSegment中的删除标记可以安全删除。
if (shouldRetainMessage(source, map, retainDeletes, entry)) {
//需要保留的消息写入writeBuffer
ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset)
stats.recopyMessage(size)
}
messagesRead += 1
} else {
// 对于压缩消息,使用深层迭代器迭代内部消息,逻辑与非压缩消息类似。
//如果整个外层消息都要保留,就不用重新压缩。
//只有部分内层消息需要保留,就对保留的内层消息重新压缩后写入writeBuffer。
val messages = ByteBufferMessageSet.deepIterator(entry)
var writeOriginalMessageSet = true
val retainedMessages = new mutable.ArrayBuffer[MessageAndOffset]
messages.foreach { messageAndOffset =>
messagesRead += 1
if (shouldRetainMessage(source, map, retainDeletes, messageAndOffset))
retainedMessages += messageAndOffset
else writeOriginalMessageSet = false
}
// There are no messages compacted out, write the original message set back
if (writeOriginalMessageSet)
ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset)
else
compressMessages(writeBuffer, entry.message.compressionCodec, messageFormatVersion, retainedMessages)
}
}