Hudi表组织结构
表组织结构如下所示:

这里我们取一张真实的hudi表的文件结构做说明
Hudi表空间
Hudi表存储的物理结构称为hudi表空间,是探索hudi表最直接的入口。
首先,根据官网spark的建表案例,创建一张hudi分区表,建表语句如下所示:
CREATE TABLE spark_hudi_table (
    ts BIGINT,
    uuid STRING,
    rider STRING,
    driver STRING,
    fare DOUBLE,
    city STRING
) USING HUDI
PARTITIONED BY (city);然后,往分区表插入数据,sql如下:
INSERT INTO spark_hudi_table
VALUES
(1695159649087,'334e26e9-8355-45cc-97c6-c31daf0df330','rider-A','driver-K',19.10,'san_francisco'),
(1695091554788,'e96c4396-3fad-413a-a942-4cb36106d721','rider-C','driver-M',27.70 ,'san_francisco'),
(1695046462179,'9909a8b1-2d15-4d3d-8ec9-efc48c536a00','rider-D','driver-L',33.90 ,'san_francisco'),
(1695332066204,'1dced545-862b-4ceb-8b43-d2a568f6616b','rider-E','driver-O',93.50,'san_francisco'),
(1695516137016,'e3cf430c-889d-4015-bc98-59bdce1e530c','rider-F','driver-P',34.15,'sao_paulo'    ),
(1695376420876,'7a84095f-737f-40bc-b62f-6b69664712d2','rider-G','driver-Q',43.40 ,'sao_paulo'    ),
(1695173887231,'3eeb61f7-c2b0-4636-99bd-5d7a5a1d2c04','rider-I','driver-S',41.06 ,'chennai'      ),
(1695115999911,'c8abbe79-8d89-47ea-b4ce-4d224bae5bfa','rider-J','driver-T',17.85,'chennai');现在,我们已经准备好了一张hudi分区表。
接下来探索这张分区表的物理结构。
基本物理结构
每一张hudi表的数据和元数据都在对应表名的空间目录下,如下所示:
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie
/data/warehouse/spark_hudi_db.db/spark_hudi_table/city=chennai
/data/warehouse/spark_hudi_db.db/spark_hudi_table/city=san_francisco
/data/warehouse/spark_hudi_db.db/spark_hudi_table/city=sao_paulo其中,".hoodie"是hudi表的元数据目录。其他三个目录都是表的分区数据目录,如果是无分区表,则表空间目录下会有许多的数据文件。

可以看到,目录深度比较大,如果以表空间位第一级目录,则上图展示的最细目录为第五级目录(.aux、.schema、.temp、archived)。
元数据目录(.hoodie)
“.hoodie”是hudi表的元数据目录,保存hudi提交信息、分区信息和元数据信息。
“.hoodie”目录下有以下文件和目录
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/.aux
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/.schema
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/.temp
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/20250928205430030.commit
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/20250928205430030.commit.requested
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/20250928205430030.inflight
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/archived
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/hoodie.properties
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata表结构和配置文件(hoodie.properties)
“hoodie.properties”文件主要是记录了表当前最新的结构和属性。
内容如下所示:
#Updated at 2025-09-28T12:54:36.663Z
#Sun Sep 28 20:54:36 CST 2025
hoodie.datasource.write.drop.partition.columns=false
hoodie.table.partition.fields=city
hoodie.table.type=COPY_ON_WRITE
hoodie.archivelog.folder=archived
hoodie.timeline.layout.version=1
hoodie.table.version=6
hoodie.table.metadata.partitions=files
hoodie.database.name=spark_hudi_db
hoodie.datasource.write.partitionpath.urlencode=false
hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator
hoodie.table.name=spark_hudi_table
hoodie.table.metadata.partitions.inflight=
hoodie.datasource.write.hive_style_partitioning=true
hoodie.table.checksum=1090855671
hoodie.table.create.schema={"type"\:"record","name"\:"spark_hudi_table_record","namespace"\:"hoodie.spark_hudi_table","fields"\:[{"name"\:"ts","type"\:["long","null"]},{"name"\:"uuid","type"\:["string","null"]},{"name"\:"rider","type"\:["string","null"]},{"name"\:"driver","type"\:["string","null"]},{"name"\:"fare","type"\:["double","null"]},{"name"\:"city","type"\:["string","null"]}]}操作提交文件({timestamp}.commit)
记录已提交的某个操作。
文件名为“{timestamp}.commit”,timestamp表示时间戳,是17位整数,精确到毫秒,例如“20250928205430030.commit”
文件内容如下所示:
{
  "partitionToWriteStats" : {
    "city=san_francisco" : [ {
      "fileId" : "527cfaf3-4a58-417a-be66-babb7888bab4-0",
      "path" : "city=san_francisco/527cfaf3-4a58-417a-be66-babb7888bab4-0_0-13-227_20250928205430030.parquet",
      "cdcStats" : null,
      "prevCommit" : "null",
      "numWrites" : 4,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 4,
      "totalWriteBytes" : 436250,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : "city=san_francisco",
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 436250,
      "minEventTime" : null,
      "maxEventTime" : null,
      "runtimeStats" : {
        "totalScanTime" : 0,
        "totalUpsertTime" : 0,
        "totalCreateTime" : 767
      }
    } ],
    "city=sao_paulo" : [ {
      "fileId" : "66f47ad1-8d7a-47ff-9751-113e02362905-0",
      "path" : "city=sao_paulo/66f47ad1-8d7a-47ff-9751-113e02362905-0_1-13-228_20250928205430030.parquet",
      "cdcStats" : null,
      "prevCommit" : "null",
      "numWrites" : 2,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 2,
      "totalWriteBytes" : 436088,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : "city=sao_paulo",
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 436088,
      "minEventTime" : null,
      "maxEventTime" : null,
      "runtimeStats" : {
        "totalScanTime" : 0,
        "totalUpsertTime" : 0,
        "totalCreateTime" : 1181
      }
    } ],
    "city=chennai" : [ {
      "fileId" : "429ecf72-48b4-43c4-bfd0-c84f4c34baf7-0",
      "path" : "city=chennai/429ecf72-48b4-43c4-bfd0-c84f4c34baf7-0_2-13-229_20250928205430030.parquet",
      "cdcStats" : null,
      "prevCommit" : "null",
      "numWrites" : 2,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 2,
      "totalWriteBytes" : 436068,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : "city=chennai",
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 436068,
      "minEventTime" : null,
      "maxEventTime" : null,
      "runtimeStats" : {
        "totalScanTime" : 0,
        "totalUpsertTime" : 0,
        "totalCreateTime" : 517
      }
    } ]
  },
  "compacted" : false,
  "extraMetadata" : {
    "schema" : "{\"type\":\"record\",\"name\":\"spark_hudi_table_record\",\"namespace\":\"hoodie.spark_hudi_table\",\"fields\":[{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"uuid\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"rider\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"driver\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"fare\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"city\",\"type\":[\"null\",\"string\"],\"default\":null}]}"
  },
  "operationType" : "INSERT"
}主要是记录了当前操作的类型,以及这次操作涉及的分区的配置信息和统计信息。
操作进行时文件({timestamp}.inflight)
记录正在进行中的某个操作。
文件名为“{timestamp}.inflight”,timestamp表示时间戳,是17位整数,精确到毫秒,例如“20250928205430030.inflight”
文件内容如下所示:
{
  "partitionToWriteStats" : {
    "city=san_francisco" : [ {
      "fileId" : "",
      "path" : null,
      "cdcStats" : null,
      "prevCommit" : "null",
      "numWrites" : 0,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 4,
      "totalWriteBytes" : 0,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : null,
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 0,
      "minEventTime" : null,
      "maxEventTime" : null,
      "runtimeStats" : null
    } ],
    "city=sao_paulo" : [ {
      "fileId" : "",
      "path" : null,
      "cdcStats" : null,
      "prevCommit" : "null",
      "numWrites" : 0,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 2,
      "totalWriteBytes" : 0,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : null,
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 0,
      "minEventTime" : null,
      "maxEventTime" : null,
      "runtimeStats" : null
    } ],
    "city=chennai" : [ {
      "fileId" : "",
      "path" : null,
      "cdcStats" : null,
      "prevCommit" : "null",
      "numWrites" : 0,
      "numDeletes" : 0,
      "numUpdateWrites" : 0,
      "numInserts" : 2,
      "totalWriteBytes" : 0,
      "totalWriteErrors" : 0,
      "tempPath" : null,
      "partitionPath" : null,
      "totalLogRecords" : 0,
      "totalLogFilesCompacted" : 0,
      "totalLogSizeCompacted" : 0,
      "totalUpdatedRecordsCompacted" : 0,
      "totalLogBlocks" : 0,
      "totalCorruptLogBlock" : 0,
      "totalRollbackBlocks" : 0,
      "fileSizeInBytes" : 0,
      "minEventTime" : null,
      "maxEventTime" : null,
      "runtimeStats" : null
    } ]
  },
  "compacted" : false,
  "extraMetadata" : { },
  "operationType" : "INSERT"
}记录操作类型和涉及的分区信息。
元数据目录(metadata)
在“.hoodie”目录下还有一层元数据目录,主要是记录
目录结构如下所示:
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/files包含两个子目录:“.hoodie”和“files”
子目录(.hoodie)
目录结构如下所示:
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/.aux
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/.schema
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/.temp
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/00000000000000010.deltacommit
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/00000000000000010.deltacommit.inflight
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/00000000000000010.deltacommit.requested
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/20250928205430030.deltacommit
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/20250928205430030.deltacommit.inflight
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/20250928205430030.deltacommit.requested
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/archived
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/.hoodie/hoodie.properties包含以下几个文件
元数据配置(hoodie.properties)
“hoodie.properties”文件记录了元数据表的配置信息。
#Properties saved on 2025-09-28T12:54:31.088Z
#Sun Sep 28 20:54:31 CST 2025
hoodie.datasource.write.drop.partition.columns=false
hoodie.table.type=MERGE_ON_READ
hoodie.archivelog.folder=archived
hoodie.populate.meta.fields=false
hoodie.compaction.payload.class=org.apache.hudi.metadata.HoodieMetadataPayload
hoodie.timeline.layout.version=1
hoodie.table.version=6
hoodie.table.base.file.format=HFILE
hoodie.table.recordkey.fields=key
hoodie.table.name=spark_hudi_table_metadata
hoodie.table.keygenerator.class=org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator
hoodie.compaction.record.merger.strategy=eeb8d96f-b1e4-49fd-bbf8-28ac514178e5
hoodie.table.checksum=963858769子目录(files)
files子目录记录了最新版本元数据信息。
目录结构如下所示:
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.1_0-0-0
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/files/.hoodie_partition_metadata
/data/warehouse/spark_hudi_db.db/spark_hudi_table/.hoodie/metadata/files/files-0000-0_0-6-5_00000000000000010.hfile包含以下文件
元数据分区文件(.hoodie_partition_metadata)
“.hoodie_partition_metadata”记录元数据表的分区信息,内容如下所示:
#partition metadata
#Sun Sep 28 20:54:34 CST 2025
commitTime=00000000000000010
partitionDepth=1包含分区深度和提交时间。
分区数据目录(partitionColumn=partitionValue)
分区数据目录是hudi分区表空间下的一个子目录,以“partitionColumn=partitionValue”的形式表示目录名称。
例如下所示
/data/warehouse/spark_hudi_db.db/spark_hudi_table/city=san_francisco/.hoodie_partition_metadata
/data/warehouse/spark_hudi_db.db/spark_hudi_table/city=san_francisco/527cfaf3-4a58-417a-be66-babb7888bab4-0_0-13-227_20250928205430030.parquet分区city=san_francisco目录下包含一个分区元数据文件和分区数据文件
分区元数据文件(.hoodie_partition_metadata)
分区的最新版本信息记录在“.hoodie_partition_metadata”文件中。
内容如下
#partition metadata
#Sun Sep 28 20:54:38 CST 2025
commitTime=20250928205430030
partitionDepth=1记录了提交时间和分区深度
分区数据文件(*.parquet)
数据文件默认是parquet,也可以设置其他的文件格式。
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;
  }