[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:
@@ -23,7 +23,6 @@ import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
@@ -49,7 +48,6 @@ import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class HoodieClusteringJob {
|
||||
|
||||
@@ -216,7 +214,7 @@ public class HoodieClusteringJob {
|
||||
}
|
||||
Option<HoodieCommitMetadata> commitMetadata = client.cluster(cfg.clusteringInstantTime, true).getCommitMetadata();
|
||||
|
||||
return handleErrors(commitMetadata.get(), cfg.clusteringInstantTime);
|
||||
return UtilHelpers.handleErrors(commitMetadata.get(), cfg.clusteringInstantTime);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -271,20 +269,7 @@ public class HoodieClusteringJob {
|
||||
LOG.info("The schedule instant time is " + instantTime.get());
|
||||
LOG.info("Step 2: Do cluster");
|
||||
Option<HoodieCommitMetadata> metadata = client.cluster(instantTime.get(), true).getCommitMetadata();
|
||||
return handleErrors(metadata.get(), instantTime.get());
|
||||
return UtilHelpers.handleErrors(metadata.get(), instantTime.get());
|
||||
}
|
||||
}
|
||||
|
||||
private int handleErrors(HoodieCommitMetadata metadata, String instantTime) {
|
||||
List<HoodieWriteStat> writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e ->
|
||||
e.getValue().stream()).collect(Collectors.toList());
|
||||
long errorsCount = writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum();
|
||||
if (errorsCount == 0) {
|
||||
LOG.info(String.format("Table imported into hoodie with %s instant time.", instantTime));
|
||||
return 0;
|
||||
}
|
||||
|
||||
LOG.error(String.format("Import failed with %d errors.", errorsCount));
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -266,8 +267,8 @@ public class HoodieCompactor {
|
||||
throw new HoodieCompactionException("There is no scheduled compaction in the table.");
|
||||
}
|
||||
}
|
||||
JavaRDD<WriteStatus> writeResponse = client.compact(cfg.compactionInstantTime);
|
||||
return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(cfg.compactionInstantTime);
|
||||
return UtilHelpers.handleErrors(compactionMetadata.getCommitMetadata().get(), cfg.compactionInstantTime);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -26,7 +26,9 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.config.DFSPropertiesConfiguration;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.util.Functions.Function1;
|
||||
@@ -303,6 +305,18 @@ public class UtilHelpers {
|
||||
return -1;
|
||||
}
|
||||
|
||||
public static int handleErrors(HoodieCommitMetadata metadata, String instantTime) {
|
||||
List<HoodieWriteStat> writeStats = metadata.getWriteStats();
|
||||
long errorsCount = writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum();
|
||||
if (errorsCount == 0) {
|
||||
LOG.info(String.format("Finish job with %s instant time.", instantTime));
|
||||
return 0;
|
||||
}
|
||||
|
||||
LOG.error(String.format("Job failed with %d errors.", errorsCount));
|
||||
return -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a factory for creating connections to the given JDBC URL.
|
||||
*
|
||||
|
||||
Reference in New Issue
Block a user