[HUDI-2435][BUG]Fix clustering handle errors (#3666)
* done * remove unused imports * code reviewed * code reviewed Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
@@ -19,16 +19,20 @@
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Async clustering client for Spark datasource.
|
||||
@@ -47,8 +51,11 @@ public class HoodieSparkClusteringClient<T extends HoodieRecordPayload> extends
|
||||
public void cluster(HoodieInstant instant) throws IOException {
|
||||
LOG.info("Executing clustering instance " + instant);
|
||||
SparkRDDWriteClient<T> writeClient = (SparkRDDWriteClient<T>) clusteringClient;
|
||||
JavaRDD<WriteStatus> res = writeClient.cluster(instant.getTimestamp(), true).getWriteStatuses();
|
||||
if (res != null && res.collect().stream().anyMatch(WriteStatus::hasErrors)) {
|
||||
Option<HoodieCommitMetadata> commitMetadata = writeClient.cluster(instant.getTimestamp(), true).getCommitMetadata();
|
||||
Stream<HoodieWriteStat> hoodieWriteStatStream = commitMetadata.get().getPartitionToWriteStats().entrySet().stream().flatMap(e ->
|
||||
e.getValue().stream());
|
||||
long errorsCount = hoodieWriteStatStream.mapToLong(HoodieWriteStat::getTotalWriteErrors).sum();
|
||||
if (errorsCount > 0) {
|
||||
// TODO: Should we treat this fatal and throw exception?
|
||||
LOG.error("Clustering for instant (" + instant + ") failed with write errors");
|
||||
}
|
||||
|
||||
@@ -24,10 +24,11 @@ import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
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.Option;
|
||||
@@ -36,13 +37,13 @@ import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.jetbrains.annotations.TestOnly;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class HoodieClusteringJob {
|
||||
|
||||
@@ -189,9 +190,9 @@ public class HoodieClusteringJob {
|
||||
private int doCluster(JavaSparkContext jsc) throws Exception {
|
||||
String schemaStr = getSchemaFromLatestInstant();
|
||||
try (SparkRDDWriteClient<HoodieRecordPayload> client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) {
|
||||
JavaRDD<WriteStatus> writeResponse =
|
||||
client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses();
|
||||
return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse);
|
||||
Option<HoodieCommitMetadata> commitMetadata = client.cluster(cfg.clusteringInstantTime, true).getCommitMetadata();
|
||||
|
||||
return handleErrors(commitMetadata.get(), cfg.clusteringInstantTime);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -230,10 +231,22 @@ public class HoodieClusteringJob {
|
||||
|
||||
LOG.info("The schedule instant time is " + instantTime.get());
|
||||
LOG.info("Step 2: Do cluster");
|
||||
JavaRDD<WriteStatus> writeResponse =
|
||||
(JavaRDD<WriteStatus>) client.cluster(instantTime.get(), true).getWriteStatuses();
|
||||
return UtilHelpers.handleErrors(jsc, instantTime.get(), writeResponse);
|
||||
Option<HoodieCommitMetadata> metadata = client.cluster(instantTime.get(), true).getCommitMetadata();
|
||||
return 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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user