1
0

[HUDI-2214]residual temporary files after clustering are not cleaned up (#3335)

This commit is contained in:
xiarixiaoyao
2021-07-27 01:26:20 +08:00
committed by GitHub
parent a5638b995b
commit 5353243449
2 changed files with 27 additions and 1 deletions

View File

@@ -53,6 +53,7 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade;
@@ -370,7 +371,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
} catch (IOException e) {
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
}
new MarkerFiles(table, clusteringCommitTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
if (clusteringTimer != null) {
long durationInMs = metrics.getDurationInMs(clusteringTimer.stop());
try {