Druid 实时数据摄入与持久化全链路深度分析
一、概述
本文档基于 Apache Druid 源码,系统梳理了从数据写入(Appenderator.add) → 内存持久化(IndexMergerV9.persist) → 多 Segment 合并(mergeQueryableIndex) → 数据字典合并(DictionaryMergingIterator) 的完整数据处理链路。涵盖的核心类包括:
| 类名 | 职责 |
|---|---|
AppenderatorImpl | 数据摄入入口,管理 Sink 和内存/持久化生命周期 |
Sink / FireHydrant | 内存中的数据容器,管理 IncrementalIndex |
IndexMergerV9 | 将内存索引持久化为 V9 格式 Segment 文件 |
IncrementalIndexAdapter | 将 IncrementalIndex 适配为 IndexableAdapter 接口 |
DictionaryEncodedColumnMerger | 字典编码列的合并器(字典合并 + 编码转换) |
DictionaryMergingIterator | 多路归并排序算法,合并多个排序字典 |
StringDimensionMergerV9 | 字符串维度的具体合并实现 |
FileSmoosher | 将多个逻辑文件合并为少量物理文件 |
二、整体架构流程
InputRow 数据行
│
▼
AppenderatorImpl.add()
│
├── Sink 不存在 → 创建新 Sink + IncrementalIndex
│
├── sink.add() → 写入 IncrementalIndex
│
├── 检查持久化触发条件
│ │
│ ├── 触发 → persistAll() → persistHydrant()
│ │ │
│ │ ▼
│ │ IndexMergerV9.persist()
│ │ │
│ │ ▼
│ │ IncrementalIndexAdapter 适配
│ │ │
│ │ ▼
│ │ makeIndexFiles() → 生成 V9 Segment 文件
│ │
│ └── 未触发 → 返回 AppenderatorAddResult
│
▼
需要 push 时
│
▼
mergeAndPush()
│
▼
IndexMergerV9.mergeQueryableIndex()
│
▼
多 Segment 字典合并 (DictionaryMergingIterator)
│
▼
生成最终合并 Segment → push 到 Deep Storage
三、阶段一:数据写入(AppenderatorImpl.add)
3.1 方法签名
public AppenderatorAddResult add(
final SegmentIdWithShardSpec identifier, // 目标 segment 标识
final InputRow row, // 输入数据行
@Nullable final Supplier<Committer> committerSupplier, // 提交元数据
final boolean allowIncrementalPersists // 是否允许自动持久化
) throws IndexSizeExceededException, SegmentNotWritableException
3.2 核心流程
-
错误检查:
throwPersistErrorIfExists()检查前次异步持久化是否有错误 -
获取或创建 Sink:
getOrCreateSink(identifier)- 如果 Sink 不存在,创建新的
Sink(interval, schema, shardSpec, ...) - 通过
segmentAnnouncer.announceSegment()公告新 Segment - 放入
sinksMap 中
- 如果 Sink 不存在,创建新的
-
写入数据:
sink.add(row, !allowIncrementalPersists)- Sink 内部委托给
IncrementalIndex.add(row)完成实际写入 - 返回
IncrementalIndexAddResult(包含 rowCount、bytesInMemory)
- Sink 内部委托给
-
更新内存统计:
rowsCurrentlyInMemory、bytesCurrentlyInMemory、totalRows -
检查持久化触发条件(见下表)
-
返回结果:
AppenderatorAddResult(identifier, numRows, isPersistRequired)
3.3 持久化触发条件
| 条件 | 说明 |
|---|---|
!sink.canAppendRow() | Sink 的当前 IncrementalIndex 已满,无法继续添加行 |
System.currentTimeMillis() > nextFlush | 超过 intermediatePersistPeriod 配置的时间窗口 |
rowsCurrentlyInMemory >= maxRowsInMemory | 内存中的行数超过 maxRowsInMemory 配置 |
bytesCurrentlyInMemory >= maxBytesInMemory | 内存中的字节数超过 maxBytesInMemory 配置 |
3.4 内存溢出保护
当 bytesCurrentlyInMemory - bytesToBePersisted > maxBytesTuningConfig 时,即使持久化后仍然超出内存限制,说明中间持久化产生的开销(如 Memory Mapped Segment)已经占满了可用内存,此时会抛出 RuntimeException 终止任务。可通过 skipBytesInMemoryOverheadCheck=true 禁用此检查。
四、阶段二:持久化流程(persistAll → persistHydrant)
4.1 persistAll 流程
-
遍历所有 sinks,收集需要持久化的 hydrants
- 对于每个 Sink,获取 hydrants 列表
limit = sink.isWritable() ? hydrants.size()-1 : hydrants.size()- 收集
hydrants[0..limit)中未 swap 的 hydrant - 如果
sink.swappable(),执行sink.swap()创建新 IncrementalIndex
-
异步持久化每个 hydrant
- 提交到
persistExecutor线程池 - 调用
indexMerger.persist(hydrant.getIndex(), interval, outDir, ...) - 根据模式决定后续处理(见下表)
- 提交到
-
完成后处理
committer.run()执行提交回调writeCommit(commitHydrants, commitMetadata)写入commit.json- 更新统计:
rowsCurrentlyInMemory -= numPersistedRows
4.2 关键设计:实时 vs 批量模式
| 特性 | 实时模式 (isOpenSegments=true) | 批量模式 (isOpenSegments=false) |
|---|---|---|
| 持久化后加载 | 立即 indexIO.loadIndex() 为 QueryableIndexSegment | 不加载,仅记录文件路径 |
| 内存占用 | Memory Mapped Segment 持续占用内存 | 最小内存占用 |
| 查询支持 | 持久化后立即可查询 | 不支持查询,直到 merge 阶段 |
| merge 时行为 | 直接使用已加载的 QueryableIndex | 需要重新 indexIO.loadIndex() |
五、阶段三:IndexMergerV9.persist 数据处理
5.1 调用链路
persist() → multiphaseMerge() → merge() → makeIndexFiles()
由于 persist 只有单个 IncrementalIndex,multiphaseMerge 直接走单阶段路径。
5.2 makeIndexFiles 完整处理流程
① 元数据合并
Metadata.merge(metadataList, combiningMetricAggs)
将各 adapter 的元数据合并为统一的 segment 元数据。
② 写入版本和工厂文件
version.bin:写入V9_VERSION = 9factory.json:写入MMappedQueryableSegmentizerFactory(默认)或自定义 SegmentLoader
③ 合并列能力 & 创建维度处理器
mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities);
Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities);
- 遍历所有 adapter 的维度和指标,合并
ColumnCapabilities - 根据合并后的能力创建对应的
DimensionHandler(String/Long/Float/Double) - 为每个维度创建
DimensionMergerV9
④ 维度字典转换
writeDimValuesAndSetupDimConversion(adapters, progress, mergedDimensions, mergers);
对每个维度调用 merger.writeMergedValueDictionary(adapters):
- 获取各 Segment 的排序字典
- 使用
DictionaryMergingIterator多路归并合并字典 - 构建
dimConversions映射(旧 dictId → 新 dictId) - 设置编码值写入器
setupEncodedValueWriter()
⑤ 构建行迭代器
TimeAndDimsIterator timeAndDimsIterator = makeMergedTimeAndDimsIterator(
adapters, mergedDimensions, mergedMetrics, rowMergerFn, handlers, mergers
);
- 为每个 adapter 创建
TransformableRowIterator - 如果维度/指标顺序不一致,通过
makeRowIteratorWithReorderedColumns()重排列 - 通过
IndexMerger.toMergedIndexRowIterator()应用字典转换 - 根据 rollup 配置选择
RowCombiningTimeAndDimsIterator(合并)或MergingRowIterator(不合并)
⑥ 遍历行写入列数据
mergeIndexesAndWriteColumns(adapters, progress, timeAndDimsIterator, timeWriter, metricWriters, mergers, fillRowNumConversions);
逐行遍历合并后的迭代器:
timeWriter.serialize(timestamp)— 写入时间戳metricWriters.get(i).serialize(metricValue)— 写入各指标值merger.processMergedRow(dimSelector)— 写入各维度编码值- 同时构建
rowNumConversions(旧行号 → 新行号映射),用于后续倒排索引合并
⑦ 构建倒排索引 & 写入列文件
makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec);
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, ...);
for (DimensionMergerV9 merger : mergers) {
merger.writeIndexes(rowNumConversions); // 构建倒排位图
ColumnDescriptor columnDesc = merger.makeColumnDescriptor();
makeColumn(v9Smoosher, dimName, columnDesc);
}
⑧ 写入索引元数据
makeIndexBinary()→index.drd:列名列表 + 时间区间 + 位图工厂类型makeMetadataBinary()→metadata.drd:聚合器 + 粒度 + Rollup 标志v9Smoosher.close()→ 合并为 smoosh 文件
5.3 列类型与序列化器对应关系
| 列类型 | 序列化器 | Serde | 数据结构 |
|---|---|---|---|
| 时间列 (__time) | LongColumnSerializer / V2 | LongNumericColumnPartSerde / V2 | 压缩 Long 数组 |
| Long 指标 | LongColumnSerializer / V2 | LongNumericColumnPartSerde / V2 | 压缩 Long 数组 |
| Float 指标 | FloatColumnSerializer / V2 | FloatNumericColumnPartSerde / V2 | 压缩 Float 数组 |
| Double 指标 | DoubleColumnSerializer / V2 | DoubleNumericColumnPartSerde / V2 | 压缩 Double 数组 |
| Complex 指标 | ComplexColumnSerializer | ComplexColumnPartSerde | GenericIndexed |
| String 维度 | DictionaryEncodedColumnMerger | DictionaryEncodedColumnPartSerde | 字典 + 编码值 + 倒排位图 |
| 数值维度 | NumericDimensionMergerV9 | 同指标列 | 同指标列 |
Null 处理双模式:
NullHandling.replaceWithDefault()为 true 时使用 V1 Serde(null 替换为默认值),为 false 时使用 V2 Serde(显式 null 位图)。
5.4 输出的 Segment 文件结构
outDir/
├── version.bin ← 版本号 (9)
├── factory.json ← 段加载器工厂
├── meta.smoosh ← smoosh 元数据 (文件名→偏移量映射)
├── 00000.smoosh ← 合并后的数据文件,包含:
│ ├── __time ← 时间列
│ ├── dim_city ← 维度列 (字典+编码值+倒排位图)
│ ├── metric_count ← 指标列
│ ├── index.drd ← 索引元数据 (列名+时间区间+位图工厂)
│ └── metadata.drd ← 段元数据 (聚合器+粒度+Rollup标志)
└── 00001.smoosh ← (如果数据超过单文件大小限制)
六、阶段四:多 Segment 数据字典合并
6.1 问题背景
每个 Segment 的字符串维度列都有独立的字典编码。例如:
Segment A 的 "city" 字典: {0: "Beijing", 1: "Shanghai", 2: "Shenzhen"}
Segment B 的 "city" 字典: {0: "Beijing", 1: "Guangzhou", 2: "Shanghai"}
合并时必须将这些独立的局部字典合并为全局统一字典,并建立旧 dictId → 新 dictId 的映射关系。
6.2 合并入口:writeMergedValueDictionary()
DictionaryEncodedColumnMerger.writeMergedValueDictionary() 方法:
第一步:收集各 Segment 的局部字典
Indexed<T>[] dimValueLookups = new Indexed[adapters.size() + 1];
for (int i = 0; i < adapters.size(); i++) {
Indexed<T> dimValues = adapters.get(i).getDimValueLookup(dimensionName);
if (dimValues != null && !allNull(dimValues)) {
dimValueLookups[i] = dimValues;
numMergeIndex++;
}
}
- 数组大小是
adapters.size() + 1,多出的一个位置留给可能需要补充的 null 值字典 - 对于
IncrementalIndexAdapter,返回indexer.getSortedIndexedValues()(已排序) - 对于
QueryableIndexIndexableAdapter,返回DictionaryEncodedColumn的字典(已排序)
第二步:处理缺失维度的 null 补充
if (convertMissingValues && !hasNull) {
dimValueLookups[adapters.size()] = getNullDimValue();
numMergeIndex++;
}
第三步:根据 numMergeIndex 选择合并策略
if (numMergeIndex > 1) {
// 多个字典 → 使用 DictionaryMergingIterator 多路归并
dictionaryMergeIterator = new DictionaryMergingIterator<>(dimValueLookups, comparator, true);
writeDictionary(() -> dictionaryMergeIterator);
for (int i = 0; i < adapters.size(); i++) {
if (dimValueLookups[i] != null && dictionaryMergeIterator.needConversion(i)) {
dimConversions.set(i, dictionaryMergeIterator.conversions[i]);
}
}
} else if (numMergeIndex == 1) {
// 只有一个字典 → 直接写入,无需转换
writeDictionary(dimValueLookup);
}
6.3 核心算法:DictionaryMergingIterator
DictionaryMergingIterator 实现了经典的多路归并排序(K-way Merge Sort) :
构造函数 — 初始化优先队列和转换缓冲区
public DictionaryMergingIterator(Indexed<T>[] dimValueLookups, Comparator comparator, boolean useDirect) {
pQueue = new PriorityQueue<>(dimValueLookups.length, comparator);
conversions = new IntBuffer[dimValueLookups.length];
for (int i = 0; i < conversions.length; i++) {
if (dimValueLookups[i] == null) continue;
// 为每个 segment 分配 DirectIntBuffer,大小 = 该 segment 字典大小
final ByteBuffer conversionDirectBuffer = ByteBuffer.allocateDirect(indexed.size() * Integer.BYTES);
conversions[i] = conversionDirectBuffer.asIntBuffer();
// 将每个 segment 的字典迭代器放入优先队列
final PeekingIterator<T> iter = Iterators.peekingIterator(indexed.iterator());
if (iter.hasNext()) {
pQueue.add(Pair.of(i, iter));
}
}
}
next() — 每次取出全局最小值并记录映射
public T next() {
// 1. 从优先队列取出当前最小值
Pair<Integer, PeekingIterator<T>> smallest = pQueue.remove();
final T value = writeTranslate(smallest, counter);
// 2. 检查其他 segment 是否有相同值(去重)
while (!pQueue.isEmpty() && Objects.equals(value, pQueue.peek().rhs.peek())) {
writeTranslate(pQueue.remove(), counter);
}
// 3. 全局 dictId 递增
counter++;
return value;
}
writeTranslate() — 记录映射关系
protected T writeTranslate(Pair<Integer, PeekingIterator<T>> smallest, int counter) {
final int index = smallest.lhs; // segment 索引
final T value = smallest.rhs.next(); // 消费当前值
// 关键:在 conversions[segmentIndex] 中记录 旧dictId(隐式position) → 新dictId(counter)
conversions[index].put(counter);
// 如果该 segment 还有更多值,放回优先队列
if (smallest.rhs.hasNext()) {
pQueue.add(smallest);
}
return value;
}
needConversion() — 判断是否需要转换
protected boolean needConversion(int index) {
IntBuffer readOnly = conversions[index].asReadOnlyBuffer();
readOnly.rewind();
int i = 0;
while (readOnly.hasRemaining()) {
if (i != readOnly.get()) {
return true; // 只要有一个位置 旧dictId ≠ 新dictId,就需要转换
}
i++;
}
return false; // 所有位置都相同,不需要转换(恒等映射)
}
6.4 具体示例演算
假设合并 3 个 Segment 的 "city" 维度:
| Segment A | Segment B | Segment C | |
|---|---|---|---|
| 字典 | {0:null, 1:"Beijing", 2:"Shanghai"} | {0:null, 1:"Guangzhou", 2:"Shanghai"} | {0:"Beijing", 1:"Shenzhen"} |
归并过程:
| 轮次 | 取出 | 值 | 全局dictId | conversions[A] | conversions[B] | conversions[C] |
|---|---|---|---|---|---|---|
| 1 | A+B | null | 0 | [0, , ] | [0, , ] | [ , ] |
| 2 | A+C | "Beijing" | 1 | [0,1,_] | [0, , ] | [1,_] |
| 3 | B | "Guangzhou" | 2 | [0,1,_] | [0,2,_] | [1,_] |
| 4 | A+B | "Shanghai" | 3 | [0,1,3] | [0,2,3] | [1,_] |
| 5 | C | "Shenzhen" | 4 | [0,1,3] | [0,2,3] | [1,4] |
最终结果:
-
全局字典:
{0:null, 1:"Beijing", 2:"Guangzhou", 3:"Shanghai", 4:"Shenzhen"} -
转换映射:
- A:
[0,1,3]→ needConversion=true(位置2: 2≠3) - B:
[0,2,3]→ needConversion=true(位置1: 1≠2) - C:
[1,4]→ needConversion=true(位置0: 0≠1)
- A:
6.5 转换应用
在行遍历阶段,convertSortedSegmentRowValuesToMergedRowValues() 通过 dimConversions 将每行的局部 dictId 实时转换为全局 dictId:
public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues(int segmentIndex, ColumnValueSelector source) {
IntBuffer converter = dimConversions.get(segmentIndex);
if (converter == null) {
return source; // 不需要转换,直接返回原选择器
}
// 包装一个新的 DimensionSelector,在 getRow() 时实时转换 dictId
return new DimensionSelector() {
IndexedInts convertedRow = new IndexedInts() {
public int get(int index) {
return converter.get(sourceDimensionSelector.getRow().get(index));
// converter.get(旧dictId) → 新dictId
}
};
};
}
6.6 倒排索引合并
字典合并后,还需要合并各 Segment 的倒排位图索引:
for (int dictId = 0; dictId < dictionarySize; dictId++) {
// 1. 通过 IndexSeeker 将全局 dictId 映射回各 segment 的本地 dictId
int seekedDictId = dictIdSeeker[j].seek(dictId);
// 2. 获取该 segment 中该 dictId 对应的位图
BitmapValues bitmapValues = adapters.get(j).getBitmapValues(dimensionName, seekedDictId);
// 3. 通过 rowNumConversions 将旧行号转换为合并后的行号
values = new ConvertingBitmapValues(bitmapValues, segmentRowNumConversions.get(j));
// 4. 合并所有 segment 的位图(升序合并,去重)
mergedIndexes = mergeAscending(convertedIterators);
// 5. 写入合并后的位图
bitmapWriter.write(bmpFactory.makeImmutableBitmap(mergedIndexes));
}
七、多阶段合并(multiphaseMerge)
7.1 触发条件
当 maxColumnsToMerge 不为 UNLIMITED_MAX_COLUMNS_TO_MERGE(-1)时,启用多阶段合并。
7.2 分阶段策略
getMergePhases() 方法将 indexes 按列数限制分组:
// 始终至少合并两个 segment,不管列数限制
if (currentPhase.size() > 1 && currentColumnCount + indexColumnCount > maxColumnsToMerge) {
toMerge.add(currentPhase);
currentPhase = new ArrayList<>();
currentPhase.add(index);
}
7.3 多层级合并
第1层: [A,B,C,D,E,F] → 分组 → [A,B], [C,D], [E,F]
↓ ↓ ↓
AB CD EF
第2层: [AB, CD, EF] → 分组 → [AB,CD], [EF]
↓ ↓
ABCD EF
第3层: [ABCD, EF] → 最终合并 → ABCDEF
每层的中间结果写入临时目录,最终层写入目标 outDir。
八、关键设计要点总结
8.1 存储设计
| 设计 | 说明 |
|---|---|
| 列式存储 | 数据按列独立序列化,每列有独立的压缩策略和编码方式 |
| 字典编码 | String 维度使用全局排序字典 + 整数编码,大幅减少存储空间 |
| 倒排索引 | 为每个字典值构建 Bitmap 倒排索引,支持高效过滤查询 |
| Smoosh 文件 | 通过 FileSmoosher 将多个逻辑文件合并为少量物理文件,减少文件句柄开销 |
8.2 内存管理
| 机制 | 说明 |
|---|---|
| 行数限制 | maxRowsInMemory 控制内存中最大行数 |
| 字节数限制 | maxBytesInMemory 控制内存中最大字节数 |
| 时间窗口 | intermediatePersistPeriod 定期触发持久化 |
| 溢出保护 | 持久化后仍超限则抛异常终止任务 |
| Direct Buffer | 字典合并使用堆外内存(DirectByteBuffer),避免占用 JVM 堆 |
8.3 合并优化
| 优化 | 说明 |
|---|---|
| 恒等映射检测 | needConversion() 检测如果映射是恒等的则跳过转换,避免不必要的间接层 |
| 多阶段合并 | multiphaseMerge 支持按 maxColumnsToMerge 分批合并,避免一次打开过多文件 |
| Null 补充 | 维度在某些 Segment 中缺失时,自动补充 null 字典值确保正确合并 |
| 算法复杂度 | 多路归并排序 O(N·logK),N=总字典值数,K=Segment 数量 |
8.4 容错设计
| 机制 | 说明 |
|---|---|
| commit.json | 记录已持久化的 hydrant 数量,支持从磁盘恢复 |
| persistError 传播 | 异步持久化错误通过 persistError 变量传播到主线程 |
| 文件锁 | basePersistDirLock 防止多个任务同时写入同一目录 |
| push 重试 | 推送到 Deep Storage 时支持 5 次重试 |
| 空列兼容 | index.drd 中空列信息放在末尾,旧版本 Historical 可安全忽略 |
九、涉及的核心源码文件
| 文件路径 | 说明 |
|---|---|
server/.../appenderator/AppenderatorImpl.java | 数据摄入入口,管理 Sink 生命周期 |
server/.../appenderator/StreamAppenderator.java | 流式摄入的 Appenderator 实现 |
processing/.../segment/IndexMergerV9.java | V9 格式的索引合并/持久化核心实现 |
processing/.../segment/IndexMerger.java | 索引合并接口定义 |
processing/.../segment/DictionaryEncodedColumnMerger.java | 字典编码列的合并器 |
processing/.../segment/DictionaryMergingIterator.java | 多路归并排序字典合并算法 |
processing/.../segment/StringDimensionMergerV9.java | 字符串维度合并实现 |
processing/.../segment/incremental/IncrementalIndex.java | 内存中的增量索引 |
processing/.../segment/incremental/IncrementalIndexAdapter.java | IncrementalIndex 到 IndexableAdapter 的适配器 |
processing/.../segment/QueryableIndexIndexableAdapter.java | QueryableIndex 到 IndexableAdapter 的适配器 |
processing/.../segment/IndexIO.java | 索引文件的读写 IO 操作 |