n3nash commented on a change in pull request #651: Spark Stage retry handling
URL: https://github.com/apache/incubator-hudi/pull/651#discussion_r279573539
##########
File path: hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java
##########
@@ -263,20 +269,135 @@ public HoodieActiveTimeline getActiveTimeline() {
* @param stats List of HoodieWriteStats
* @throws HoodieIOException if some paths can't be finalized on storage
*/
- public void finalizeWrite(JavaSparkContext jsc, List<HoodieWriteStat> stats)
+ public void finalizeWrite(JavaSparkContext jsc, String instantTs,
List<HoodieWriteStat> stats)
throws HoodieIOException {
- if (config.isConsistencyCheckEnabled()) {
- List<String> pathsToCheck = stats.stream()
- .map(stat -> stat.getTempPath() != null
- ? stat.getTempPath() : stat.getPath())
- .collect(Collectors.toList());
-
- List<String> failingPaths = new ConsistencyCheck(config.getBasePath(),
pathsToCheck, jsc,
- config.getFinalizeWriteParallelism())
- .check(MAX_CONSISTENCY_CHECKS,
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS);
- if (failingPaths.size() > 0) {
- throw new HoodieIOException("Could not verify consistency of paths : "
+ failingPaths);
+ rollbackFailedWrites(jsc, instantTs, stats,
config.isConsistencyCheckEnabled());
+ }
+
+ /**
+ * Reconciles WriteStats and marker files to detect and safely delete
duplicate data files created because of Spark
+ * retries.
+ *
+ * @param jsc Spark Context
+ * @param instantTs Instant Timestamp
+ * @param stats Hoodie Write Stat
+ * @param consistencyCheckEnabled Consistency Check Enabled
+ * @throws HoodieIOException
+ */
+ protected void rollbackFailedWrites(JavaSparkContext jsc, String instantTs,
List<HoodieWriteStat> stats,
+ boolean consistencyCheckEnabled) throws HoodieIOException {
+ try {
+ // Reconcile marker and data files with WriteStats so that partially
written data-files due to failed
+ // (but succeeded on retry) tasks are removed.
+ String basePath = getMetaClient().getBasePath();
+ FileSystem fs = getMetaClient().getFs();
+ Path markerDir = new Path(metaClient.getMarkerFolderPath(instantTs));
+
+ if (!fs.exists(markerDir)) {
+ // Happens when all writes are appends
+ return;
}
+
+ List<String> invalidDataPaths = FSUtils.getAllDataFilesForMarkers(fs,
basePath, instantTs, markerDir.toString());
+ List<String> validDataPaths = stats.stream().map(w ->
String.format("%s/%s", basePath, w.getPath()))
+ .filter(p -> p.endsWith(".parquet")).collect(Collectors.toList());
+ // Contains list of partially created files. These needs to be cleaned
up.
+ invalidDataPaths.removeAll(validDataPaths);
+ logger.warn("InValid data paths=" + invalidDataPaths);
+
+ Map<String, List<Pair<String, String>>> groupByPartition =
invalidDataPaths.stream()
+ .map(dp -> Pair.of(new Path(dp).getParent().toString(), dp))
+ .collect(Collectors.groupingBy(Pair::getKey));
+
+ if (!groupByPartition.isEmpty()) {
+ // Ensure all files in delete list is actually present. This is
mandatory for an eventually consistent FS.
+ // Otherwise, we may miss deleting such files. If files are not found
even after retries, fail the commit
+ if (consistencyCheckEnabled) {
+ // This will either ensure all files to be deleted are present.
+ boolean checkPassed =
+ jsc.parallelize(new ArrayList<>(groupByPartition.values()),
config.getFinalizeWriteParallelism())
+ .map(partitionWithFileList -> {
+ final FileSystem fileSystem = metaClient.getFs();
+ if (partitionWithFileList.isEmpty()) {
Review comment:
Change this to use java stream semantics to avoid these if checks ?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services