[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:
@@ -39,7 +39,6 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.FlinkHoodieIndexFactory;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
@@ -68,7 +67,6 @@ import org.apache.hadoop.conf.Configuration;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
@@ -346,23 +344,20 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
@Override
|
||||
public void commitCompaction(
|
||||
String compactionInstantTime,
|
||||
List<WriteStatus> writeStatuses,
|
||||
Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
HoodieCommitMetadata metadata,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
HoodieFlinkTable<T> table = getHoodieTable();
|
||||
HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(
|
||||
table, compactionInstantTime, HoodieList.of(writeStatuses), config.getSchema());
|
||||
extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
|
||||
completeCompaction(metadata, writeStatuses, table, compactionInstantTime);
|
||||
completeCompaction(metadata, table, compactionInstantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completeCompaction(
|
||||
HoodieCommitMetadata metadata,
|
||||
List<WriteStatus> writeStatuses,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
|
||||
String compactionCommitTime) {
|
||||
this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction");
|
||||
List<HoodieWriteStat> writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList());
|
||||
List<HoodieWriteStat> writeStats = metadata.getWriteStats();
|
||||
final HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime);
|
||||
try {
|
||||
this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty());
|
||||
@@ -391,16 +386,11 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
|
||||
protected HoodieWriteMetadata<List<WriteStatus>> compact(String compactionInstantTime, boolean shouldComplete) {
|
||||
// only used for metadata table, the compaction happens in single thread
|
||||
try {
|
||||
List<WriteStatus> writeStatuses =
|
||||
getHoodieTable().compact(context, compactionInstantTime).getWriteStatuses();
|
||||
commitCompaction(compactionInstantTime, writeStatuses, Option.empty());
|
||||
return writeStatuses;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error while compacting instant: " + compactionInstantTime);
|
||||
}
|
||||
HoodieWriteMetadata<List<WriteStatus>> compactionMetadata = getHoodieTable().compact(context, compactionInstantTime);
|
||||
commitCompaction(compactionInstantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
return compactionMetadata;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Reference in New Issue
Block a user