Hudi表文件组织结构
以下是文件的物理结构。
以如下建表语句创建一张表。
create table hudi_mor_table (
id bigint,
name string,
price double,
ts bigint) using hudi
tblproperties (
type = 'mor',
primaryKey = 'id',
preCombineField = 'ts');
下面是一张Hudi MOR表的结构
[root@abc-10-10-10-11 ~]# hadoop fs -ls /data/hive/warehouse/hudi_test_db.db/hudi_mor_table
Found 8 items
-rw-r--r-- 3 hadoop supergroup 923 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.2a333780-bdbc-4e7f-9e23-29cc3df065e5-0_20250602113213286.log.1_0-14-607
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie
-rw-r--r-- 3 hadoop supergroup 96 2025-06-02 11:29 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie_partition_metadata
-rw-r--r-- 3 hadoop supergroup 435004 2025-06-02 11:30 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/2a333780-bdbc-4e7f-9e23-29cc3df065e5-0_0-14-607_20250602113042199.parquet
-rw-r--r-- 3 hadoop supergroup 435042 2025-06-02 11:31 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/2a333780-bdbc-4e7f-9e23-29cc3df065e5-0_0-14-607_20250602113127526.parquet
-rw-r--r-- 3 hadoop supergroup 435077 2025-06-02 11:32 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/2a333780-bdbc-4e7f-9e23-29cc3df065e5-0_0-14-607_20250602113213286.parquet
-rw-r--r-- 3 hadoop supergroup 434967 2025-06-02 11:29 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/2a333780-bdbc-4e7f-9e23-29cc3df065e5-0_0-23-615_20250602112853402.parquet
-rw-r--r-- 3 hadoop supergroup 435081 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/2a333780-bdbc-4e7f-9e23-29cc3df065e5-0_0-29-817_20250602113317028.parquet
表分区元数据(.hoodie_partition_metadata)
下面是一张分区表的分区元数据
[root@abc-10-10-10-11 ~]# hadoop fs -cat /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie_partition_metadata
#partition metadata
#Mon Jun 2 11:29:07 CST 2025
commitTime=20250602112853402
partitionDepth=0
包含最近一次提交时间。
表数据(.hoodie)
表的数据存在于.hoodie目录。
[root@abc-10-10-10-11 ~]# hadoop fs -ls /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie
Found 24 items
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/.aux
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/.schema
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/.temp
-rw-r--r-- 3 hadoop supergroup 1403 2025-06-02 11:29 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602112853402.deltacommit
-rw-r--r-- 3 hadoop supergroup 798 2025-06-02 11:29 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602112853402.deltacommit.inflight
-rw-r--r-- 3 hadoop supergroup 0 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602112853402.deltacommit.requested
-rw-r--r-- 3 hadoop supergroup 1416 2025-06-02 11:31 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113042199.deltacommit
-rw-r--r-- 3 hadoop supergroup 798 2025-06-02 11:30 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113042199.deltacommit.inflight
-rw-r--r-- 3 hadoop supergroup 0 2025-06-02 11:30 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113042199.deltacommit.requested
-rw-r--r-- 3 hadoop supergroup 1416 2025-06-02 11:31 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113127526.deltacommit
-rw-r--r-- 3 hadoop supergroup 798 2025-06-02 11:31 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113127526.deltacommit.inflight
-rw-r--r-- 3 hadoop supergroup 0 2025-06-02 11:31 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113127526.deltacommit.requested
-rw-r--r-- 3 hadoop supergroup 1416 2025-06-02 11:32 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113213286.deltacommit
-rw-r--r-- 3 hadoop supergroup 798 2025-06-02 11:32 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113213286.deltacommit.inflight
-rw-r--r-- 3 hadoop supergroup 0 2025-06-02 11:32 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113213286.deltacommit.requested
-rw-r--r-- 3 hadoop supergroup 1706 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113257533.deltacommit
-rw-r--r-- 3 hadoop supergroup 1520 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113257533.deltacommit.inflight
-rw-r--r-- 3 hadoop supergroup 0 2025-06-02 11:32 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113257533.deltacommit.requested
-rw-r--r-- 3 hadoop supergroup 1416 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113317028.commit
-rw-r--r-- 3 hadoop supergroup 0 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113317028.compaction.inflight
-rw-r--r-- 3 hadoop supergroup 2459 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/20250602113317028.compaction.requested
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/archived
-rw-r--r-- 3 hadoop supergroup 1139 2025-06-02 11:29 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/hoodie.properties
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata
元数据目录(metadata)
元数据目录包含两个次级目录
[root@abc-10-10-10-11 ~]# hadoop fs -ls /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata
Found 2 items
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/.hoodie
drwxr-xr-x - hadoop supergroup 0 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/files
files
files中记录了最新版本元数据信息
[root@abc-10-10-10-11 ~]# hadoop fs -ls /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/files
Found 3 items
-rw-r--r-- 3 hadoop supergroup 80246 2025-06-02 11:33 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.1_0-0-0
-rw-r--r-- 3 hadoop supergroup 96 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/files/.hoodie_partition_metadata
-rw-r--r-- 3 hadoop supergroup 6807 2025-06-02 11:28 /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/files/files-0000-0_0-6-5_00000000000000010.hfile
分区版本信息(.hoodie_partition_metadata)
分区的最新版本信息记录在.hoodie_partition_metadata文件中。
[root@abc-10-10-10-11 ~]# hadoop fs -cat /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/metadata/files/.hoodie_partition_metadata
#partition metadata
#Mon Jun 2 11:28:57 CST 2025
commitTime=00000000000000010
partitionDepth=1
属性(hoodie.properties)
hoodie.properties中记录了表当前最新的结构和属性。
[root@abc-10-10-10-11 ~]# hadoop fs -cat /data/hive/warehouse/hudi_test_db.db/hudi_mor_table/.hoodie/hoodie.properties
#Updated at 2025-06-02T03:28:59.657Z
#Mon Jun 2 11:28:59 CST 2025
hoodie.table.keygenerator.class=org.apache.hudi.keygen.NonpartitionedKeyGenerator
hoodie.table.precombine.field=ts
hoodie.table.version=6
hoodie.database.name=hudi_test_db
hoodie.datasource.write.hive_style_partitioning=true
hoodie.table.metadata.partitions.inflight=
hoodie.table.checksum=2605833246
hoodie.table.create.schema={"type"\:"record","name"\:"hudi_mor_table_record","namespace"\:"hoodie.hudi_mor_table","fields"\:[{"name"\:"id","type"\:["long","null"]},{"name"\:"name","type"\:["string","null"]},{"name"\:"price","type"\:["double","null"]},{"name"\:"ts","type"\:["long","null"]}]}
hoodie.archivelog.folder=archived
hoodie.table.name=hudi_mor_tbl
hoodie.compaction.payload.class=org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
hoodie.compaction.record.merger.strategy=eeb8d96f-b1e4-49fd-bbf8-28ac514178e5
hoodie.table.type=MERGE_ON_READ
hoodie.datasource.write.partitionpath.urlencode=false
hoodie.datasource.write.drop.partition.columns=false
hoodie.table.metadata.partitions=files
hoodie.timeline.layout.version=1
hoodie.table.recordkey.fields=id
Hudi文件管理
Hudi将记录写入数据 parquet文件或日志 log文件。
文件组——HoodieFileGroup
HoodieFileGroup包含的字段如下
package org.apache.hudi.common.model;
public class HoodieFileGroup implements Serializable {
// 文件组ID
private final HoodieFileGroupId fileGroupId;
// FileSlices,按照提交时间大小排序
private final TreeMap <String, FileSlice> fileSlices;
// 时间轴
private final HoodieTimeline timeline;
// 上一次完成的Instant,充当水位基准.
private final Option <HoodieInstant> lastInstant;
}
HoodieFileGroupId是HoodieFileGroup的唯一标识。
每个 HoodieFileGroup中会包含一个 TreeMap<CommitTime,FileSlice>,按照 CommitTime从大到小排序。
为方便操作会保存一个 Timeline,以及最后完成的 Instant。
生成文件组
下面以 AbstractTableFileSystemView#buildFileGroups为例,分析 HoodieFileGroup的生成逻辑
protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieBaseFile> baseFileStream,
Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
// 获取所有数据文件对应的分区路径、文件ID(相同的分区路径、文件ID会对应数据文件列表)
Map<Pair<String, String>, List<HoodieBaseFile>> baseFiles =
baseFileStream.collect(Collectors.groupingBy(baseFile -> {
String partitionPathStr = getPartitionPathFor(baseFile);
return Pair.of(partitionPathStr, baseFile.getFileId());
}));
// 获取所有日志文件对应的分区路径、文件ID(相同的分区路径、文件ID会对应日志文件列表)
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = logFileStream.collect(Collectors.groupingBy((logFile) -> {
String partitionPathStr =
FSUtils.getRelativePartitionPath(metaClient.getBasePathV2(), logFile.getPath().getParent());
return Pair.of(partitionPathStr, logFile.getFileId());
}));
// 初始化所有的数据文件和日志文件(过滤掉相同的<Partition, FileID>)
Set<Pair<String, String>> fileIdSet = new HashSet<>(baseFiles.keySet());
fileIdSet.addAll(logFiles.keySet());
List<HoodieFileGroup> fileGroups = new ArrayList<>();
fileIdSet.forEach(pair -> {
// 获取文件ID
String fileId = pair.getValue();
String partitionPath = pair.getKey();
// 生成新的文件组
HoodieFileGroup group = new HoodieFileGroup(partitionPath, fileId, timeline);
if (baseFiles.containsKey(pair)) { // 包含在数据文件集合中
// 添加该数据文件
baseFiles.get(pair).forEach(group::addBaseFile);
}
if (logFiles.containsKey(pair)) { // 包含在日志文件集合中
// 添加该日志文件
logFiles.get(pair).forEach(group::addLogFile);
}
if (addPendingCompactionFileSlice) { // 添加pending的compaction的FileSlice
Option<Pair<String, CompactionOperation>> pendingCompaction =
getPendingCompactionOperationWithInstant(group.getFileGroupId());
if (pendingCompaction.isPresent()) { // 存在pending的compaction
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
// so that any new ingestion uses the correct base-instant
// 添加至文件组
group.addNewFileSliceAtInstant(pendingCompaction.get().getKey());
}
}
fileGroups.add(group);
});
return fileGroups;
}
对于文件组的构建,首先会对指定分区的所有数据文件和日志文件进行一次排序(按照分区路径和文件ID),然后对每个 <分区路径,文件ID>生成一个文件组,并将具有相同 <分支路径,文件ID>的日志文件和数据文件放入该文件组。
文件组ID——HoodieFileGroupId
每个文件组ID由分区路径和文件ID唯一标识,不同的分区或不同的文件ID均属于不同的 HoodieFileGroup。
package org.apache.hudi.common.model;
public class HoodieFileGroupId implements Serializable {
// 分区路径
private final String partitionPath;
// 文件ID
private final String fileId;
}
文件片——FileSlice
一个 FileSlice对应一个数据文件和日志文件列表,并且其包含一个基准时间(数据文件和日志文件都有相同的时间基准)。
package org.apache.hudi.common.model;
public class FileSlice implements Serializable {
/**
* File Group Id of the Slice.文件组ID
*/
private final HoodieFileGroupId fileGroupId;
/**
* Point in the timeline, at which the slice was created.Instant的时间
*/
private final String baseInstantTime;
/**
* data file, with the compacted data, for this slice.数据文件
*/
private HoodieBaseFile baseFile;
/**
* List of appendable log files with real time data - Sorted with greater log version first - Always empty for
* copy_on_write storage.日志文件列表,按照更低版本排序,在MOR时存在,COW时为空
*/
private final TreeSet<HoodieLogFile> logFiles;
}
数据文件——HoodieBaseFile
每个数据文件包含了一个文件状态,文件的全路径以及文件的长度。
package org.apache.hudi.common.model;
public class HoodieBaseFile extends BaseFile {
// 文件状态
private transient FileStatus fileStatus;
// 文件全路径
private final String fullPath;
// 文件名
private final String fileName;
// 文件大小
private long fileLen;
private Option<BaseFile> bootstrapBaseFile;
}
日志文件——HoodieLogFile
日志文件与数据文件包含信息类似,日志文件的初始化版本为1。
package org.apache.hudi.common.model;
public class HoodieLogFile implements Serializable {
// 日志文件扩展名
public static final String DELTA_EXTENSION = ".log";
// 日志文件基准版本
public static final Integer LOGFILE_BASE_VERSION = 1;
// 文件状态
private transient FileStatus fileStatus;
// 文件路径
private final String pathStr;
// 文件大小
private long fileLen;
}
日志文件名生成
使用文件ID、提交时间、日志文件扩展名、版本号、writeToken生成完整的文件名。
而对于文件ID、提交时间等可直接按照 _进行分割或正则表达式来获取。
public static String makeLogFileName(String fileId, String logFileExtension, String baseCommitTime, int version,
String writeToken) {
String suffix = (writeToken == null)
? String.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version)
: String.format("%s_%s%s.%d_%s", fileId, baseCommitTime, logFileExtension, version, writeToken);
return LOG_FILE_PREFIX + suffix;
}