1
0

[HUDI-2641] Avoid deleting all inflight commits heartbeats while rolling back failed writes (#3956)

This commit is contained in:
Udit Mehrotra
2021-11-18 05:33:50 -08:00
committed by GitHub
parent 24def0b30d
commit 4e067ca581
5 changed files with 156 additions and 32 deletions

View File

@@ -852,18 +852,13 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS,
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
rollbackFailedBootstrap();
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instant, config);
break;
} else {
rollback(instant, skipLocking);
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instant, config);
}
}
// Delete any heartbeat files for already rolled back commits
try {
HeartbeatUtils.cleanExpiredHeartbeats(this.heartbeatClient.getAllExistingHeartbeatInstants(),
createMetaClient(true), basePath);
} catch (IOException io) {
LOG.error("Unable to delete heartbeat files", io);
}
}
protected List<String> getInstantsToRollback(HoodieTableMetaClient metaClient, HoodieFailedWritesCleaningPolicy cleaningPolicy, Option<String> curInstantTime) {

View File

@@ -21,7 +21,6 @@ package org.apache.hudi.client.heartbeat;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
@@ -30,9 +29,6 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Helper class to delete heartbeat for completed or failed instants with expired heartbeats.
@@ -55,6 +51,8 @@ public class HeartbeatUtils {
deleted = fs.delete(new Path(heartbeatFolderPath + Path.SEPARATOR + instantTime), false);
if (!deleted) {
LOG.error("Failed to delete heartbeat for instant " + instantTime);
} else {
LOG.info("Deleted the heartbeat for instant " + instantTime);
}
} catch (IOException io) {
LOG.error("Unable to delete heartbeat for instant " + instantTime, io);
@@ -63,20 +61,19 @@ public class HeartbeatUtils {
}
/**
* Deletes the heartbeat files for instants with expired heartbeats without any active instant.
* @param allExistingHeartbeatInstants
* @param metaClient
* @param basePath
* Deletes the heartbeat file for the specified instant.
* @param fs Hadoop FileSystem instance
* @param basePath Hoodie table base path
* @param instantTime Commit instant time
* @param config HoodieWriteConfig instance
* @return Boolean indicating whether heartbeat file was deleted or not
*/
public static void cleanExpiredHeartbeats(List<String> allExistingHeartbeatInstants,
HoodieTableMetaClient metaClient, String basePath) {
Set<String> nonExpiredHeartbeatInstants = metaClient.getActiveTimeline()
.filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
allExistingHeartbeatInstants.stream().forEach(instant -> {
if (!nonExpiredHeartbeatInstants.contains(instant)) {
deleteHeartbeatFile(metaClient.getFs(), basePath, instant);
}
});
public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String instantTime, HoodieWriteConfig config) {
if (config.getFailedWritesCleanPolicy().isLazy()) {
return deleteHeartbeatFile(fs, basePath, instantTime);
}
return false;
}
/**