前言
本文重点介绍Paimon压缩参数full-compaction.delta-commits的整体调用流程,涉及到CompactTask的详情:Paimon源码解读 -- Compaction-1.MergeTreeCompactTask
一.full-compaction.delta-commits 参数源码分析
1️.参数定义(CoreOptions)
java
public static final ConfigOption<Integer> FULL_COMPACTION_DELTA_COMMITS =
key("full-compaction.delta-commits")
.intType()
.noDefaultValue() // 没有默认值
.withDescription(
"Full compaction will be constantly triggered after delta commits.");
关键信息:
- 参数类型 :
Integer - 默认值 :无(
noDefaultValue()) - 含义:经过指定数量的 delta 提交后,触发一次全量压缩
2.入口FlinkSink.createWriteProvider()
CASE-1: write-only = true,后续走专用压缩任务
CASE-2: write-only = false,走自动压缩
- 优先使用
full-compaction.delta-commits参数,其次使用compaction.optimization-interval参数
deltaCommits参数取值
- 等于full-compaction.delta-commits
- 等于(compaction.optimization-interval / cp间隔)取整
- 先判断是否配置了
changelog-producer = full-compaction或 配置了任意上面两个参数,走GlobalFullCompactionSinkWrite - 再判断是否配置了
force-lookup = true 且 lookup-wait = false才走异步AsyncLookupSinkWrite
最后兜底StoreSinkWriteImpl
java
private StoreSinkWrite.Provider createWriteProvider(
CheckpointConfig checkpointConfig, boolean isStreaming, boolean hasSinkMaterializer) {
SerializableRunnable assertNoSinkMaterializer =
() ->
Preconditions.checkArgument(
!hasSinkMaterializer,
String.format(
"Sink materializer must not be used with Paimon sink. "
+ "Please set '%s' to '%s' in Flink's config.",
ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE
.key(),
ExecutionConfigOptions.UpsertMaterialize.NONE.name()));
Options options = table.coreOptions().toConfiguration();
ChangelogProducer changelogProducer = table.coreOptions().changelogProducer();
boolean waitCompaction;
CoreOptions coreOptions = table.coreOptions();
// CASE-1: write-only = true,后续走专用压缩任务
if (coreOptions.writeOnly()) {
waitCompaction = false;
}
// CASE-2: write-only = false,走自动压缩
else {
waitCompaction = coreOptions.prepareCommitWaitCompaction();
int deltaCommits = -1;
// 优先使用 full-compaction.delta-commits 参数
if (options.contains(FULL_COMPACTION_DELTA_COMMITS)) {
deltaCommits = options.get(FULL_COMPACTION_DELTA_COMMITS);
}
// 其次使用 compaction.optimization-interval 参数
else if (options.contains(CHANGELOG_PRODUCER_FULL_COMPACTION_TRIGGER_INTERVAL)) {
long fullCompactionThresholdMs =
options.get(CHANGELOG_PRODUCER_FULL_COMPACTION_TRIGGER_INTERVAL).toMillis();
// 根据 checkpoint 间隔将时间转换为提交次数 = compaction.optimization-interval / cp间隔
deltaCommits =
(int)
(fullCompactionThresholdMs
/ checkpointConfig.getCheckpointInterval());
}
// (1) 若配置了changelog-producer = full-compaction 或 配置了上面两个参数,走GlobalFullCompactionSinkWrite
if (changelogProducer == ChangelogProducer.FULL_COMPACTION || deltaCommits >= 0) {
int finalDeltaCommits = Math.max(deltaCommits, 1);
return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> {
assertNoSinkMaterializer.run();
return new GlobalFullCompactionSinkWrite(
table,
commitUser,
state,
ioManager,
ignorePreviousFiles,
waitCompaction,
finalDeltaCommits, // 传递deltaCommits参数
isStreaming,
memoryPool,
metricGroup);
};
}
}
// (2) 若配置了force-lookup = true 且 lookup-wait = false才走异步AsyncLookupSinkWrite
if (coreOptions.needLookup() && !coreOptions.prepareCommitWaitCompaction()) {
return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> {
assertNoSinkMaterializer.run();
return new AsyncLookupSinkWrite(
table,
commitUser,
state,
ioManager,
ignorePreviousFiles,
waitCompaction,
isStreaming,
memoryPool,
metricGroup);
};
}
// CASE-1最后走这
return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> {
assertNoSinkMaterializer.run();
return new StoreSinkWriteImpl(
table,
commitUser,
state,
ioManager,
ignorePreviousFiles,
waitCompaction,
isStreaming,
memoryPool,
metricGroup);
};
}
3.GlobalFullCompactionSinkWrite提交全量压缩任务
该类的继承关系如下 
(1) prepareCommit() -- 每次cp都检查是否需要提交全量压缩任务
步骤如下
- 检查之前提交的全量压缩是否成功
- 收集当前cp阶段前所有已经写入的buckets -- 赋值给writtenBuckets
- 判断是否应该触发全量压缩
判断依据: writtenBuckets 不为空,且当前cpid % deltaCommits==0 时提交触发一次全量压缩,例如:deltaCommits=5,则在 checkpointId=5,10,15,20... 时触发
- 需要触发全量压缩,调
submitFullCompaction()去提交全量压缩任务
java
@Override
public List<Committable> prepareCommit(boolean waitCompaction, long checkpointId)
throws IOException {
// 1.检查之前提交的全量压缩是否成功
checkuccessfulFullCompaction();
// 2.收集当前cp阶段写入的buckets
if (!currentWrittenBuckets.isEmpty()) {
writtenBuckets
.computeIfAbsent(checkpointId, k -> new HashSet<>())
.addAll(currentWrittenBuckets);
currentWrittenBuckets.clear();
}
if (LOG.isDebugEnabled()) {
for (Map.Entry<Long, Set<Tuple2<BinaryRow, Integer>>> checkpointIdAndBuckets :
writtenBuckets.entrySet()) {
LOG.debug(
"Written buckets for checkpoint #{} are:", checkpointIdAndBuckets.getKey());
for (Tuple2<BinaryRow, Integer> bucket : checkpointIdAndBuckets.getValue()) {
LOG.debug(" * partition {}, bucket {}", bucket.f0, bucket.f1);
}
}
}
// 3.判断是否应该触发全量压缩
// 判断依据: 当前writtenBuckets不为null 且 (checkpointId % deltaCommits == 0 || checkpointId == Long.MAX_VALUE)
// isFullCompactedIdentifier()作用: 每隔 deltaCommits 次提交触发一次全量压缩
// 例如:deltaCommits=5,则在 checkpointId=5,10,15,20... 时触发
if (!writtenBuckets.isEmpty() && isFullCompactedIdentifier(checkpointId, deltaCommits)) {
waitCompaction = true;
}
// 4.调`submitFullCompaction()`去提交全量压缩任务
if (waitCompaction) {
if (LOG.isDebugEnabled()) {
LOG.debug("Submit full compaction for checkpoint #{}", checkpointId);
}
submitFullCompaction(checkpointId);
commitIdentifiersToCheck.add(checkpointId);
}
return super.prepareCommit(waitCompaction, checkpointId);
}
(2) submitFullCompaction()
它其实还是调父类StoreSinkWriteImpl的属性TableWriteImpl对每个bucket桶执行全量压缩操作
java
private void submitFullCompaction(long currentCheckpointId) {
if (LOG.isDebugEnabled()) {
LOG.debug("Submit full compaction for checkpoint #{}", currentCheckpointId);
}
// compactedBuckets存储当次压缩任务要压缩的buckets是哪些
Set<Tuple2<BinaryRow, Integer>> compactedBuckets = new HashSet<>();
// 遍历所有历史写入的 buckets
writtenBuckets.forEach(
(checkpointId, buckets) -> {
for (Tuple2<BinaryRow, Integer> bucket : buckets) {
if (compactedBuckets.contains(bucket)) {
continue; // 避免重复压缩
}
compactedBuckets.add(bucket);
try {
// 调父类StoreSinkWriteImpl的属性TableWriteImpl对每个bucket桶执行全量压缩操作
write.compact(bucket.f0, bucket.f1, true);// true表示执行fullCompaction
} catch (Exception e) {
throw new RuntimeException(e);
}
}
});
}
4.TableWriteImpl.compact()
作用:调FileStoreWrite的实现类的compact()
java
@Override
public void compact(BinaryRow partition, int bucket, boolean fullCompaction) throws Exception {
// 这里的write是FileStoreWrite的实现类,如KeyValueFileStoreWrite
write.compact(partition, bucket, fullCompaction);
}
5.FileStoreWrite实现类的compact()
这是个接口,其实现类如下 
下面看AbstractFileStoreWrite实现的comapct(),涉及KeyValueFileStoreWrite详情请看Paimon源码解读 -- KeyValueFileStoreWrite
步骤如下:
- 先调
getWriterWrapper()获取对应partition-bucket的WriterContainer(其实就是RecordWriter包装起来) - 用其的
RecordWriter实现类如MergeTreeWriter的compact()进行压缩,并传入fullCompaction表示是否需要全量压缩
java
@Override
public void compact(BinaryRow partition, int bucket, boolean fullCompaction) throws Exception {
// 1.先调getWriterWrapper()获取对于partition-bucket的WriterContainer
// 2.用其中的MergeTreeWriter实现类的compact方法进行压缩,并传入fullCompaction是否需要全量压缩
getWriterWrapper(partition, bucket).writer.compact(fullCompaction);
}
// 调用的getWriterWrapper()
protected WriterContainer<T> getWriterWrapper(BinaryRow partition, int bucket) {
// 1.获取当前对应partition的buckets
Map<Integer, WriterContainer<T>> buckets = writers.get(partition);
if (buckets == null) {
buckets = new HashMap<>();
writers.put(partition.copy(), buckets);
}
// 2.为每个bucket创建WriterContainer对象(其实就是RecordWriter包装起来)
return buckets.computeIfAbsent(
bucket, k -> createWriterContainer(partition.copy(), bucket, ignorePreviousFiles));
}
6.RecordWriter实现类的compact()
这是个接口,其实现类如下

下面以MergeTreeWriter为例
(1) compact()
java
@Override
public void compact(boolean fullCompaction) throws Exception {
// 调flushWriteBuffer(true, fullCompaction)
flushWriteBuffer(true, fullCompaction);
}
(2) flushWriteBuffer() -- 核心流程
这是触发压缩的核心流程,其步骤如下
- 检查是否需要等待最近的压缩
- 如果配置的是write-only=true,则不需要等待;
- 其他情况,则进一步判断,当前levels的Sorted Run文件数 > num-sorted-run.stop-trigger比较,则需要等待压缩;否则,不需要
- 用
KeyValueFileWriterFactory去创建的Writer(changelog、Merge) - 将
writeBuffer缓冲区的数据写入到文件中,用创建的writer去写入到L0层 - 调
writer.result()收集写入后的DataFileMeta文件数据并将这些文件添加到CompactManager实现类,以便后续压缩执行 - 调
trySyncLatestCompaction(waitForLatestCompaction)等待之前的压缩任务完成 - 调
CompactManager实现类的triggerCompaction()去触发新压缩,携带是否全量压缩参数,详情请看Paimon源码解读 -- CompactManager
java
/**
* 压缩的核心触发代码,compact()中调用该方法
* @param waitForLatestCompaction: 是否等待最近的压缩完成
* @param forcedFullCompaction: 是否全量压缩
* @throws Exception
*/
private void flushWriteBuffer(boolean waitForLatestCompaction, boolean forcedFullCompaction)
throws Exception {
if (writeBuffer.size() > 0) {
// 1.检查是否需要等待最近的压缩
// 如果配置的是write-only=true,则不需要等待;
// 其他情况,则进一步判断,当前levels的Sorted Run文件数 > num-sorted-run.stop-trigger比较,则需要等待压缩;否则,不需要
if (compactManager.shouldWaitForLatestCompaction()) {
waitForLatestCompaction = true;
}
// 2.用KeyValueFileWriterFactory去创建的Writer
// changelog-producer = input 且 并不是只写入+I,则需要changelogWriter去生成changelog
final RollingFileWriter<KeyValue, DataFileMeta> changelogWriter =
(changelogProducer == ChangelogProducer.INPUT && !isInsertOnly)
? writerFactory.createRollingChangelogFileWriter(0)
: null;
// 创建dataWriter,以便写入数据
final RollingFileWriter<KeyValue, DataFileMeta> dataWriter =
writerFactory.createRollingMergeTreeFileWriter(0, FileSource.APPEND);
try {
// 3.将writeBuffer缓冲区的数据写入到文件中,用writer去写入到L0层
writeBuffer.forEach(
keyComparator,
mergeFunction,
changelogWriter == null ? null : changelogWriter::write,
dataWriter::write);
} finally {
writeBuffer.clear();
if (changelogWriter != null) {
changelogWriter.close();
}
dataWriter.close();
}
// 4.收集写入后的DataFileMeta文件数据
List<DataFileMeta> dataMetas = dataWriter.result();
if (changelogWriter != null) {
newFilesChangelog.addAll(changelogWriter.result());
} else if (changelogProducer == ChangelogProducer.INPUT && isInsertOnly) {
List<DataFileMeta> changelogMetas = new ArrayList<>();
for (DataFileMeta dataMeta : dataMetas) {
String newFileName = writerFactory.newChangelogFileName(0);
DataFileMeta changelogMeta = dataMeta.rename(newFileName);
writerFactory.copyFile(dataMeta, changelogMeta);
changelogMetas.add(changelogMeta);
}
newFilesChangelog.addAll(changelogMetas);
}
// 将新文件添加到 compactManager
for (DataFileMeta dataMeta : dataMetas) {
newFiles.add(dataMeta);
compactManager.addNewFile(dataMeta);
}
}
// 5.等待之前的压缩任务完成
trySyncLatestCompaction(waitForLatestCompaction);
// 6.调CompactManager实现类的triggerCompaction()去触发新压缩,携带是否全量压缩参数
compactManager.triggerCompaction(forcedFullCompaction);
}
二.总结
(1) write-only = true
用NoopCompactManager(),不执行任何压缩操作,而是由后续专用压缩任务去执行相应的压缩操作
(2) write-only = false
write-only = false,走自动压缩,用MergeTreeCompactManager
- 优先使用
full-compaction.delta-commits参数,其次使用compaction.optimization-interval参数
deltaCommits参数取值
- 等于full-compaction.delta-commits
- 等于(compaction.optimization-interval / cp间隔)取整
- 先判断是否配置了
changelog-producer = full-compaction或 配置了任意上面两个参数,走GlobalFullCompactionSinkWrite - 在判断是否配置了
force-lookup = true 且 lookup-wait = false才走异步AsyncLookupSinkWrite
最后兜底StoreSinkWriteImpl
以GlobalFullCompactionSinkWrite为例
读取所有文件 -> Merge -> 写入新文件