1
0

[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:
zhangyue19921010
2021-10-13 06:24:48 +08:00
committed by GitHub
parent 8a487eafa7
commit e6711b171a
2 changed files with 30 additions and 10 deletions

View File

@@ -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");
}