1
0

[HUDI-3370] The files recorded in the commit may not match the actual ones for MOR Compaction (#4753)

* use HoodieCommitMetadata to replace writeStatuses computation

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
YueZhang
2022-02-14 11:12:52 +08:00
committed by GitHub
parent 55777fec05
commit 76e2faa28d
17 changed files with 129 additions and 114 deletions

View File

@@ -20,12 +20,15 @@ package org.apache.hudi.sink.compact;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.CleanFunction;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.action.compact.CompactHelpers;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.StreamerUtil;
@@ -147,8 +150,11 @@ public class CompactionCommitSink extends CleanFunction<CompactionCommitEvent> {
.flatMap(Collection::stream)
.collect(Collectors.toList());
HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(
table, instant, HoodieList.of(statuses), writeClient.getConfig().getSchema());
// commit the compaction
this.writeClient.commitCompaction(instant, statuses, Option.empty());
this.writeClient.commitCompaction(instant, metadata, Option.empty());
// Whether to clean up the old log file when compaction
if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) {