Hudi压缩(compaction)实现分析

1、介绍

在上一篇文章 Hudi聚簇(Clustering)实现分析 中介绍过 Hudi 提供了一种数据重组方式 Clustering,用于对 CopyOnWrite 存储类型的表文件进行合并,该合并主要是针对 parquet 格式的文件进行重排,但对于 MergeOnRead 表类型,Hudi 为提高流式写入速度,直接以 Avro 格式向文件系统中写入 log 文件,然后提供一种数据压缩的的方式对 log 文件进行压缩,如果合并计划中只含有 log 文件,则将 log 文件压缩成 parquet 文件,然后开始写新的 log 文件,如果合并计划同时包含 log 文件和 parquet 文件,则将两种格式的文件进行压缩生成新的 parquet 文件,然后开始写新的 log 文件,Hudi 将这种压缩方式称为 Compaction。

2、源码分析

2.1 生成HoodieCompactionPlan

Hudi 中所有生成 plan 的逻辑都由 BaseHoodieTableServiceClient 的 scheduleTableServiceInternal 控制,包括 archieve/cluster/compact/log_compat/clean 等服务。

// org/apache/hudi/client/BaseHoodieTableServiceClient.java
  protected Option<String> scheduleTableServiceInternal(String instantTime, Option<Map<String, String>> extraMetadata,
                                                        TableServiceType tableServiceType) {
    if (!tableServicesEnabled(config)) {
      return Option.empty();
    }

    Option<String> option = Option.empty();
    HoodieTable<?, ?, ?, ?> table = createTable(config, hadoopConf);

    // Flink/Spark schedule service 的入口
    switch (tableServiceType) {
      case ARCHIVE:
        LOG.info("Scheduling archiving is not supported. Skipping.");
        break;
      case CLUSTER:
        LOG.info("Scheduling clustering at instant time :" + instantTime);
        Option<HoodieClusteringPlan> clusteringPlan = table
            .scheduleClustering(context, instantTime, extraMetadata);
        option = clusteringPlan.isPresent() ? Option.of(instantTime) : Option.empty();
        break;
      case COMPACT:
        LOG.info("Scheduling compaction at instant time :" + instantTime);
        Option<HoodieCompactionPlan> compactionPlan = table
            .scheduleCompaction(context, instantTime, extraMetadata);
        option = compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty();
        break;
      case LOG_COMPACT:
        LOG.info("Scheduling log compaction at instant time :" + instantTime);
        Option<HoodieCompactionPlan> logCompactionPlan = table
            .scheduleLogCompaction(context, instantTime, extraMetadata);
        option = logCompactionPlan.isPresent() ? Option.of(instantTime) : Option.empty();
        break;
      case CLEAN:
        LOG.info("Scheduling cleaning at instant time :" + instantTime);
        Option<HoodieCleanerPlan> cleanerPlan = table
            .scheduleCleaning(context, instantTime, extraMetadata);
        option = cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty();
        break;
      default:
        throw new IllegalArgumentException("Invalid TableService " + tableServiceType);
    }

    Option<String> instantRange = delegateToTableServiceManager(tableServiceType, table);
    if (instantRange.isPresent()) {
      LOG.info("Delegate instant [" + instantRange.get() + "] to table service manager");
    }

    return option;
  }

前期会进行一些校验工作,判断当前 Timeline 与生成 plan 的 instant 的合法性,然后执行 scheduleCompaction,底层会调用到 ScheduleCompactionActionExecutor 执行器,该执行器负责生成 HoodieCompactionPlan,plan 生成成功最后 saveToCompactionRequested 创建 *.compaction.requested 文件。

// org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java
  public Option<HoodieCompactionPlan> execute() {
    ValidationUtils.checkArgument(this.table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
        "Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
            + this.table.getMetaClient().getTableType().name());
    // 判断并发写模式是否为 OPTIMISTIC_CONCURRENCY_CONTROL,清理策略是否为 LAZY
    // 默认情况下 if 条件为 true,会进行一些校验工作
    if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()
        && !config.getFailedWritesCleanPolicy().isLazy()) {
      // 校验,如果是 Spark 引擎,如果有inflight状态的写入,那么最早的instant的时间一定大于正在进行压缩的时间
      // TODO(yihua): this validation is removed for Java client used by kafka-connect.  Need to revisit this.
      if (config.getEngineType() == EngineType.SPARK) {
        // if there are inflight writes, their instantTime must not be less than that of compaction instant time
        table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction().firstInstant()
            .ifPresent(earliestInflight -> ValidationUtils.checkArgument(
                HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime),
                "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
                    + ", Compaction scheduled at " + instantTime));
      }
      // 校验,commit、deltacommit、compaction类型的instant的时间一定小于正在进行压缩的时间
      // Committed and pending compaction instants should have strictly lower timestamps
      List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
          .getWriteTimeline().filterCompletedAndCompactionInstants().getInstantsAsStream()
          .filter(instant -> HoodieTimeline.compareTimestamps(
              instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
          .collect(Collectors.toList());
      ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
          "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"
              + conflictingInstants);
    }

    // 开始生成 compaction plan
    HoodieCompactionPlan plan = scheduleCompaction();
    Option<HoodieCompactionPlan> option = Option.empty();
    // 根据 COMPACT_ACTION 还是 LOG_COMPACTION_ACTION 类型序列化保存 requested 状态
    if (plan != null && nonEmpty(plan.getOperations())) {
      extraMetadata.ifPresent(plan::setExtraMetadata);
      try {
        if (operationType.equals(WriteOperationType.COMPACT)) {
          HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED,
              HoodieTimeline.COMPACTION_ACTION, instantTime);
          table.getActiveTimeline().saveToCompactionRequested(compactionInstant,
              TimelineMetadataUtils.serializeCompactionPlan(plan));
        } else {
          HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED,
              HoodieTimeline.LOG_COMPACTION_ACTION, instantTime);
          table.getActiveTimeline().saveToLogCompactionRequested(logCompactionInstant,
              TimelineMetadataUtils.serializeCompactionPlan(plan));
        }
      } catch (IOException ioe) {
        throw new HoodieIOException("Exception scheduling compaction", ioe);
      }
      option = Option.of(plan);
    }

    return option;
  }

scheduleCompaction 执行前会调用 needCompact() 方法,主要是判断当前的 CompactionTriggerStrategy 策略是否满足生成 plan 的条件,策略包括 NUM_COMMITS/NUM_COMMITS_AFTER_LAST_REQUEST/TIME_ELAPSED/NUM_AND_TIME/NUM_OR_TIME 集中,默认的策略是 NUM_COMMITS,判断是否能够生成 plan 的条件分为两种:

  • Timeline上之前已经有完成的compaction plan,则统计从上一次完成的 compaction plan 时间开始,到最新 deltacommit 时间之间间隔的 num_dealtacommit 个数,如果超过 hoodie.compact.inline.max.delta.commits值则尝试生成 plan。
  • Timeline上没有完成的compaction plan,则统计第一次 deltacommit 时间到最新 deltacommit 时间之间间隔的 num_deltacommit 个数,如果超过 hoodie.compact.inline.max.delta.commits值则尝试生成 plan。
// org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java
  private HoodieCompactionPlan scheduleCompaction() {
    LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
    // 判断 deltacommit 间隔是否满足 compaction 要求
    // judge if we need to compact according to num delta commits and time elapsed
    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
    if (compactable) {
      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
      try {
        context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan");
        // 生成 compaction plan 细节实现
        return planGenerator.generateCompactionPlan();
      } catch (IOException e) {
        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
      }
    }
    return new HoodieCompactionPlan();
  }

如果 needCompact 满足生成 plan 的条件,则调用 generateCompactionPlan 具体生成 plan,生成过程大致分为以下几个步骤:

  • 拿到 Hoodie 表的所有分区信息,然后根据 CompactionStrategy 策略筛选出符合条件的分区,默认的 CompactionStrategy 策略是 LogFileSizeBasedCompactionStrategy 策略,会拿所有分区信息。
  • 依次遍历每个分区,获取分区内所有最新的 FileSlice 版本,对每个 FileSlice 进行过滤,一个 FileSlice 对应构建一个 CompactionOperation,最后将生成的所有 CompactionOperation 类型的 operations 转换为 HoodieCompactionOperation 类型用于序列化传输。
  • 最后根据 LogFileSizeBasedCompactionStrategy 策略对 operations 进行排序和过滤 orderAndFilter,选出符合策略条件的 operations,将这些 operations 进行封装生成 HoodieCompactionPlan。
// org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java
  public HoodieCompactionPlan generateCompactionPlan() throws IOException {
    // Accumulator to keep track of total log files for a table
    HoodieAccumulator totalLogFiles = engineContext.newAccumulator();
    // Accumulator to keep track of total log file slices for a table
    HoodieAccumulator totalFileSlices = engineContext.newAccumulator();

    // TODO : check if maxMemory is not greater than JVM or executor memory
    // TODO - rollback any compactions in flight
    HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
    // 拿到表的所有分区信息
    List<String> partitionPaths = FSUtils.getAllPartitionPaths(engineContext, writeConfig.getMetadataConfig(), metaClient.getBasePath());

    // 根据配置的CompactionStrategy过滤出符合条件的分区,默认是 LogFileSizeBasedCompactionStrategy 策略
    // filter the partition paths if needed to reduce list status
    partitionPaths = filterPartitionPathsByStrategy(writeConfig, partitionPaths);
    if (partitionPaths.isEmpty()) {
      // In case no partitions could be picked, return no compaction plan
      return null;
    }
    LOG.info("Looking for files to compact in " + partitionPaths + " partitions");
    engineContext.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact: " + writeConfig.getTableName());

    SyncableFileSystemView fileSystemView = (SyncableFileSystemView) this.hoodieTable.getSliceView();

    // 获取当前 pending compaction/logcompaction/clustering 对应的 fileId
    // Exclude file groups under compaction.
    Set<HoodieFileGroupId> fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
        .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
        .collect(Collectors.toSet());

    // Exclude files in pending clustering from compaction.
    fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));

    // Exclude files in pending logcompaction.
    if (filterLogCompactionOperations()) {
      fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getPendingLogCompactionOperations()
          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
          .collect(Collectors.toList()));
    }

    // 最后一次 commit 完成的 instant
    String lastCompletedInstantTime = hoodieTable.getMetaClient()
        .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
            HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
        .filterCompletedInstants().lastInstant().get().getTimestamp();
    LOG.info("Last completed instant time " + lastCompletedInstantTime);
    Option<InstantRange> instantRange = CompactHelpers.getInstance().getInstantRange(metaClient);

    // 依次遍历每个分区,获取分区内最新的 FileSlice 版本,对每个 FileSlice 进行过滤,
    // 如果有属于 pending compaction/clustering 状态的 FileSlice,则忽略这部分 FileSlice,
    // 对剩下的 FileSlice 构建 CompactionOperation
    // 需要注意的是,即使前面 needCompact() 为true,这里经过 filterFileSlice 过滤还是会出现 operations 为空的情况,即没有 plan 生成
    List<HoodieCompactionOperation> operations = engineContext.flatMap(partitionPaths, partitionPath -> fileSystemView
        .getLatestFileSlices(partitionPath)
        .filter(slice -> filterFileSlice(slice, lastCompletedInstantTime, fgIdsInPendingCompactionAndClustering, instantRange))
        .map(s -> {
          List<HoodieLogFile> logFiles =
              s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(toList());
          totalLogFiles.add(logFiles.size());
          totalFileSlices.add(1L);
          // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
          // for Map operations and collecting them finally in Avro generated classes for storing
          // into meta files.6
          Option<HoodieBaseFile> dataFile = s.getBaseFile();
          // 一个 FileSlice 对应一个 CompactionOperation
          return new CompactionOperation(dataFile, partitionPath, logFiles,
              writeConfig.getCompactionStrategy().captureMetrics(writeConfig, s));
        }), partitionPaths.size()).stream()
        // CompactionOperation 转换为 HoodieCompactionOperation(avro generated)
        .map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());

    LOG.info("Total of " + operations.size() + " compaction operations are retrieved");
    LOG.info("Total number of latest files slices " + totalFileSlices.value());
    LOG.info("Total number of log files " + totalLogFiles.value());
    LOG.info("Total number of file slices " + totalFileSlices.value());

    if (operations.isEmpty()) {
      LOG.warn("No operations are retrieved for " + metaClient.getBasePath());
      return null;
    }

    // 根据构建的 HoodieCompactionOperation operations 生成 HoodieCompactionPlan,
    // 内部会根据 CompactionStrategy 策略对 operations 进行排序和过滤 orderAndFilter,选出符合策略条件的 operations
    // Filter the compactions with the passed in filter. This lets us choose most effective compactions only
    HoodieCompactionPlan compactionPlan = getCompactionPlan(metaClient, operations);
    // 校验,当前compactionPlan中的FileGroupId不属于pendingCompaction范畴,避免同一个FileGroupId文件被两个compaction plan处理
    ValidationUtils.checkArgument(
        compactionPlan.getOperations().stream().noneMatch(
            op -> fgIdsInPendingCompactionAndClustering.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
        "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
            + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactionAndClustering
            + ", Selected workload :" + compactionPlan);
    if (compactionPlan.getOperations().isEmpty()) {
      LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
    }
    return compactionPlan;
  }

2.2 执行HoodieCompactionPlan

生成完 plan,接下来就是执行该 plan,对 plan 中的多个 operation 的文件进行合并操作。合并前,还是会进行一些校验工作,判断要执行的 compaction instant 是 requested 状态还是 inflight 状态,如果是 requested 状态则跳过检测继续执行,如果是 inflight 状态,说明该 plan 之前执行失败过,需要先对该 plan 进行 rollback 回滚后再执行该 plan。

// org/apache/hudi/client/SparkRDDTableServiceClient.java
  protected HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(String compactionInstantTime, boolean shouldComplete) {
    HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
    // 获取 compaction 类型的 Timeline(包括 requested、inflight 状态)
    HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
    // 构造 inflight 状态的 instant
    HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
    // 如果 pending compaction 中包含 inflight 状态的操作,则先回滚该 inflight 的 compaction
    if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
      table.rollbackInflightCompaction(inflightInstant, commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false));
      table.getMetaClient().reloadActiveTimeline();
    }
    compactionTimer = metrics.getCompactionCtx();
    // 执行 compact 动作
    HoodieWriteMetadata<HoodieData<WriteStatus>> writeMetadata = table.compact(context, compactionInstantTime);
    // 更新 HoodieWriteMetadata 元数据信息
    HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses()));
    if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) {
      // compaction 执行完后的收尾操作
      completeTableService(TableServiceType.COMPACT, compactionMetadata.getCommitMetadata().get(), table, compactionInstantTime);
    }
    return compactionMetadata;
  }

执行compact方法会调用RunCompactionActionExecutor执行器的execute方法,这里会先做一些准备工作,比如校验 compaction plan 是否是 requested 状态、获取要执行的 compaction plan、根据 commit 记录更新 schema 信息等等,然后进行 compact 合并,合并后的结果以 HoodieData<WriteStatus> 类型封装起来,并更新到 HoodieWriteMetadata 元数据信息中。

// org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java
  public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
    // 获取pending的compaction或者log compaction
    HoodieTimeline pendingMajorOrMinorCompactionTimeline = WriteOperationType.COMPACT.equals(operationType)
        ? table.getActiveTimeline().filterPendingCompactionTimeline()
        : table.getActiveTimeline().filterPendingLogCompactionTimeline();
    // 前置校验操作
    // 如果pendingMajorOrMinorCompactionTimeline不包含instantTime的.compaction.requested计划,则直接抛异常结束
    compactor.preCompact(table, pendingMajorOrMinorCompactionTimeline, this.operationType, instantTime);

    HoodieWriteMetadata<HoodieData<WriteStatus>> compactionMetadata = new HoodieWriteMetadata<>();
    try {
      // 拿到compaction plan
      // generate compaction plan
      // should support configurable commit metadata
      HoodieCompactionPlan compactionPlan = operationType.equals(WriteOperationType.COMPACT)
          ? CompactionUtils.getCompactionPlan(table.getMetaClient(), instantTime)
          : CompactionUtils.getLogCompactionPlan(table.getMetaClient(), instantTime);

      // 读取Timeline上距离compaction最近的commit的instant对应的schema,保证读取的schema是最新的
      // try to load internalSchema to support schema Evolution
      HoodieWriteConfig configCopy = config;
      // 获取 internalSchema(LATEST_SCHEMA)和 avroSchema(SCHEMA_KEY)
      Pair<Option<String>, Option<String>> schemaPair = InternalSchemaCache
          .getInternalSchemaAndAvroSchemaForClusteringAndCompaction(table.getMetaClient(), instantTime);
      if (schemaPair.getLeft().isPresent() && schemaPair.getRight().isPresent()) {
        // should not influence the original config, just copy it
        configCopy = HoodieWriteConfig.newBuilder().withProperties(config.getProps()).build();
        // configCopy设置InternalSchema 到 hoodie.internal.schema 参数和设置 avroSchema 到 hoodie.avro.schema 参数
        // ksmou-TODO: 两者有什么区别
        configCopy.setInternalSchemaString(schemaPair.getLeft().get());
        configCopy.setSchema(schemaPair.getRight().get());
      }

      // 执行 compaction 压缩操作,并返回写入的结果状态
      HoodieData<WriteStatus> statuses = compactor.compact(
          context, compactionPlan, table, configCopy, instantTime, compactionHandler);

      // 持久化 HoodieData<WriteStatus> statuses,避免后续重复计算
      compactor.maybePersist(statuses, context, config, instantTime);
      context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata: " + config.getTableName());
      // WriteStatus 类中包含 HoodieWriteStat 对象,这里只需要 HoodieWriteStat 信息
      List<HoodieWriteStat> updateStatusMap = statuses.map(WriteStatus::getStat).collectAsList();
      HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
      for (HoodieWriteStat stat : updateStatusMap) {
        metadata.addWriteStat(stat.getPartitionPath(), stat);
      }
      // LATEST_SCHEMA -> "latest_schema", SCHEMA_KEY -> "schema"
      metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
      if (schemaPair.getLeft().isPresent()) {
        metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, schemaPair.getLeft().get());
        metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaPair.getRight().get());
      }
      // Setting operationType, which is compact.
      metadata.setOperationType(operationType);
      compactionMetadata.setWriteStatuses(statuses);
      compactionMetadata.setCommitted(false);
      compactionMetadata.setCommitMetadata(Option.of(metadata));
    } catch (Exception e) {
      throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
    }

    return compactionMetadata;
  }

合并前先将 compaction plan 从 requested 状态流转为 inflight 状态,并且对 schema 进行相应的更新,然后对 compaction plan 包含的所有 operations 调用 parallelize 方法,并行地对单个 operation 调用 compact 执行合并。

// org/apache/hudi/table/action/compact/HoodieCompactor.java
  /**
   * Execute compaction operations and report back status.
   */
  public HoodieData<WriteStatus> compact(
      HoodieEngineContext context, HoodieCompactionPlan compactionPlan,
      HoodieTable table, HoodieWriteConfig config, String compactionInstantTime,
      HoodieCompactionHandler compactionHandler) {
    if (compactionPlan == null || (compactionPlan.getOperations() == null)
        || (compactionPlan.getOperations().isEmpty())) {
      return context.emptyHoodieData();
    }
    CompactionExecutionHelper executionHelper = getCompactionExecutionStrategy(compactionPlan);

    // compaction的requested状态流转为inflight状态
    // ksmou-TODO: 如果是并行执行,两个任务处理同一个requested plan,这里状态转换会创建inflight失败
    // Transition requested to inflight file.
    executionHelper.transitionRequestedToInflight(table, compactionInstantTime);
    table.getMetaClient().reloadActiveTimeline();

    HoodieTableMetaClient metaClient = table.getMetaClient();
    TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient);

    // Here we firstly use the table schema as the reader schema to read
    // log file.That is because in the case of MergeInto, the config.getSchema may not
    // the same with the table schema.
    try {
      if (StringUtils.isNullOrEmpty(config.getInternalSchema())) {  // hoodie.internal.schema
        // 获取表的schema信息并添加到config中
        // 这里是优先从Timeline上最近的一个完成的commit信息中解析出表的schema信息,其次从Table schema或者data file中解析schema信息
        Schema readerSchema = schemaResolver.getTableAvroSchema(false);
        config.setSchema(readerSchema.toString());  // hoodie.avro.schema
        // config.getSchema(); // hoodie.avro.schema
      }
    } catch (Exception e) {
      // If there is no commit in the table, just ignore the exception.
    }

    // 生成plan时将 CompactionOperation 转换为 HoodieCompactionOperation(generateCompactionPlan)
    // 这里还原回来,将 HoodieCompactionOperation 转换为 CompactionOperation
    // Compacting is very similar to applying updates to existing file
    List<CompactionOperation> operations = compactionPlan.getOperations().stream()
        .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
    LOG.info("Compactor compacting " + operations + " files");

    String maxInstantTime = getMaxInstantTime(metaClient);

    context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices: " + config.getTableName());
    TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier();
    // if this is a MDT, set up the instant range of log reader just like regular MDT snapshot reader.
    Option<InstantRange> instantRange = CompactHelpers.getInstance().getInstantRange(metaClient);

    // 对单个 operation 进行 compact 合并
    return context.parallelize(operations).map(operation -> compact(
        compactionHandler, metaClient, config, operation, compactionInstantTime, maxInstantTime, instantRange, taskContextSupplier, executionHelper))
        .flatMap(List::iterator);
  }

对单个 operation 执行合并的过程,本质就是对 operation 内的所有log文件和base文件进行合并(base文件可能为空),首先构建 HoodieMergedLogRecordScanner 对象扫描所有log文件,将读取到的records封装到该对象中,然后对该 scanner 扫描的内容和 base 文件进行合并,如果 base 文件为空,则执行 handleInsert 方式合并,否则执行 handleUpdate 方式合并,最后将合并后的状态信息封装成 List<WriteStatus> 返回。

// org/apache/hudi/table/action/compact/HoodieCompactor.java
/**
   * Execute a single compaction operation and report back status.
   */
  public List<WriteStatus> compact(HoodieCompactionHandler compactionHandler,
                                   HoodieTableMetaClient metaClient,
                                   HoodieWriteConfig config,
                                   CompactionOperation operation,
                                   String instantTime,
                                   String maxInstantTime,
                                   Option<InstantRange> instantRange,
                                   TaskContextSupplier taskContextSupplier,
                                   CompactionExecutionHelper executionHelper) throws IOException {
    FileSystem fs = metaClient.getFs();
    Schema readerSchema;
    Option<InternalSchema> internalSchemaOption = Option.empty();
    if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
      readerSchema = new Schema.Parser().parse(config.getSchema());
      internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
      // its safe to modify config here, since we are running in task side.
      ((HoodieTable) compactionHandler).getConfig().setDefault(config);
    } else {
      readerSchema = HoodieAvroUtils.addMetadataFields(
          new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
    }
    LOG.info("Compaction operation started for base file: " + operation.getDataFileName() + " and delta files: " + operation.getDeltaFileNames()
        + " for commit " + instantTime);
    // TODO - FIX THIS
    // Reads the entire avro file. Always only specific blocks should be read from the avro file
    // (failure recover).
    // Load all the delta commits since the last compaction commit and get all the blocks to be
    // loaded and load it using CompositeAvroLogReader
    // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.

    // 计算单个 compaction operation 用到的最大 memory,用于下面扫描log文件
    long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(taskContextSupplier, config);
    LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);

    // 获取单个operation中的所有log文件
    List<String> logFiles = operation.getDeltaFileNames().stream().map(
        p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString())
        .collect(toList());
    
    // 构建HoodieMergedLogRecordScanner对象,对所有log文件进行扫描,封装到scanner中
    // 具体扫描细节交由 构建HoodieMergedLogRecordScanner#performScan 方法执行
    HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder()
        .withFileSystem(fs)
        .withBasePath(metaClient.getBasePath())
        .withLogFilePaths(logFiles)
        .withReaderSchema(readerSchema)
        .withLatestInstantTime(executionHelper.instantTimeToUseForScanning(instantTime, maxInstantTime))
        .withInstantRange(instantRange)
        .withInternalSchema(internalSchemaOption.orElse(InternalSchema.getEmptyInternalSchema()))
        .withMaxMemorySizeInBytes(maxMemoryPerCompaction)
        .withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled())
        .withReverseReader(config.getCompactionReverseLogReadEnabled())
        .withBufferSize(config.getMaxDFSStreamBufferSize())
        .withSpillableMapBasePath(config.getSpillableMapBasePath())
        .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
        .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
        .withOperationField(config.allowOperationMetadataField())
        .withPartition(operation.getPartitionPath())
        .withOptimizedLogBlocksScan(executionHelper.enableOptimizedLogBlockScan(config))
        .withRecordMerger(config.getRecordMerger())
        .withInstantRange(instantRange)
        .build();

    // 获取operation中的base文件,可能为空也可能不空
    Option<HoodieBaseFile> oldDataFileOpt =
        operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());

    // 校验,scanner为空时,如果oldDataFileOpt不存在则直接返回空,存在则继续执行compact逻辑
    // Considering following scenario: if all log blocks in this fileSlice is rollback, it returns an empty scanner.
    // But in this case, we need to give it a base file. Otherwise, it will lose base file in following fileSlice.
    if (!scanner.iterator().hasNext()) {
      if (!oldDataFileOpt.isPresent()) {
        scanner.close();
        return new ArrayList<>();
      } else {
        // TODO: we may directly rename original parquet file if there is not evolution/devolution of schema
        /*
        TaskContextSupplier taskContextSupplier = hoodieCopyOnWriteTable.getTaskContextSupplier();
        String newFileName = FSUtils.makeDataFileName(instantTime,
            FSUtils.makeWriteToken(taskContextSupplier.getPartitionIdSupplier().get(), taskContextSupplier.getStageIdSupplier().get(), taskContextSupplier.getAttemptIdSupplier().get()),
            operation.getFileId(), hoodieCopyOnWriteTable.getBaseFileExtension());
        Path oldFilePath = new Path(oldDataFileOpt.get().getPath());
        Path newFilePath = new Path(oldFilePath.getParent(), newFileName);
        FileUtil.copy(fs,oldFilePath, fs, newFilePath, false, fs.getConf());
        */
      }
    }

    // Compacting is very similar to applying updates to existing file
    Iterator<List<WriteStatus>> result;
    // 对 logFiles 和 base File 执行合并操作,如果 base File 存在执行 handleUpdate 操作,否则执行 handleInsert 操作,并返回写入结果的状态信息
    result = executionHelper.writeFileAndGetWriteStats(compactionHandler, operation, instantTime, scanner, oldDataFileOpt);
    scanner.close();
    Iterable<List<WriteStatus>> resultIterable = () -> result;
    // peek如果是个中间过程,主要用于调试,即不改变原始本身的值,而且需要有 terminal operation(类似forEach、collect等) 操作peek才会真正调用
    // 除此之外,peek() 还有另一种用途:改变元素的内部状态。"On top of that, peek() can be useful in another scenario: when we want to alter the inner state of an element."
    // forEach() 则是一个最终操作。除此之外,peek() 和 forEach() 再无其他不同。
    // 将写入状态更新到HoodieWriteStat中
    return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream).peek(s -> {
      s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
      s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
      s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
      s.getStat().setPartitionPath(operation.getPartitionPath());
      s.getStat()
          .setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
      s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
      s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
      s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
      RuntimeStats runtimeStats = new RuntimeStats();
      runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
      s.getStat().setRuntimeStats(runtimeStats);
    }).collect(toList());
  }

至此,compaction plan 的合并过程已经结束,然后调用 completeTableService 执行一些收尾工作,包括WriteState状态更新到指标、更新 metadata table 信息、Timeline 上 compaction plan 的 inflight 状态流转为 completed 状态等。

// org/apache/hudi/client/SparkRDDTableServiceClient.java
  protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime) {
    this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction: " + config.getTableName());
    List<HoodieWriteStat> writeStats = metadata.getWriteStats();  // 和 metadata.getPartitionToWriteStats(); 是等价的
    final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime);

    try {
      this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty());
      // 执行指标更新和一些清理操作
      finalizeWrite(table, compactionCommitTime, writeStats);
      // hoodie.metadata.enable=true时,更新metadata table信息
      // commit to data table after committing to metadata table.
      updateTableMetadata(table, metadata, compactionInstant);
      LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata);
      // inflight流转为completed
      CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
    } finally {
      this.txnManager.endTransaction(Option.of(compactionInstant));
    }
    WriteMarkersFactory.get(config.getMarkersType(), table, compactionCommitTime)
        .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
    if (compactionTimer != null) {
      long durationInMs = metrics.getDurationInMs(compactionTimer.stop());
      HoodieActiveTimeline.parseDateFromInstantTimeSafely(compactionCommitTime).ifPresent(parsedInstant ->
          metrics.updateCommitMetrics(parsedInstant.getTime(), durationInMs, metadata, COMPACTION_ACTION)
      );
    }
    LOG.info("Compacted successfully on commit " + compactionCommitTime);
  }

3、总结

Hudi Compaction 数据压缩为 MergeOnRead 类型的表提供了一种小文件合并操作,在生成 HoodieCompactionPlan 阶段先筛选出需要进行 compaction 的分区信息,依次遍历每个分区,获取分区下满足合并要求的 FileSlice 信息,生成一系列的 HoodieCompactionOperation,然后将其封装生成 HoodieCompactionPlan,执行 HoodieCompactionPlan 阶段会先从 plan 中拿到所有的 operations 信息,依次遍历这些 operations,对每个 operation 分别执行 compact 操作对其包含的 log 文件和 parquet 文件进行合并,合并完成后更新写入的状态信息。

  • 0
    点赞
  • 2
    收藏
    觉得还不错? 一键收藏
  • 0
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值