1
0

[HUDI-785] Refactor compaction/savepoint execution based on ActionExector abstraction (#1548)

- Savepoint and compaction classes moved to table.action.* packages
 - HoodieWriteClient#savepoint(...) returns void
 - Renamed HoodieCommitArchiveLog -> HoodieTimelineArchiveLog
 - Fixed tests to take into account the additional validation done
 - Moved helper code into CompactHelpers and SavepointHelpers
This commit is contained in:
vinoth chandar
2020-04-25 18:26:44 -07:00
committed by GitHub
parent 19cc15c098
commit 19ca0b5629
58 changed files with 789 additions and 601 deletions

View File

@@ -42,7 +42,7 @@ import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.compact.OperationResult;
import org.apache.hudi.table.action.compact.OperationResult;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hadoop.fs.FSDataInputStream;

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.spark.api.java.JavaSparkContext;
@@ -79,12 +80,13 @@ public class SavepointsCommand implements CommandMarker {
String result;
try (JavaSparkContext jsc = SparkUtil.initJavaSparkConf("Create Savepoint")) {
HoodieWriteClient client = createHoodieClient(jsc, metaClient.getBasePath());
if (client.savepoint(commitTime, user, comments)) {
try {
client.savepoint(commitTime, user, comments);
// Refresh the current
refreshMetaClient();
result = String.format("The commit \"%s\" has been savepointed.", commitTime);
} else {
result = String.format("Failed: Could not savepoint commit \"%s\".", commitTime);
} catch (HoodieSavepointException se) {
result = String.format("Failed: Could not create savepoint \"%s\".", commitTime);
}
}
return result;

View File

@@ -27,7 +27,7 @@ import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.compact.strategy.UnBoundedCompactionStrategy;
import org.apache.hudi.table.action.compact.strategy.UnBoundedCompactionStrategy;
import org.apache.hudi.utilities.HDFSParquetImporter;
import org.apache.hudi.utilities.HDFSParquetImporter.Config;
import org.apache.hudi.utilities.HoodieCleaner;
@@ -283,10 +283,11 @@ public class SparkMain {
private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
if (client.restoreToSavepoint(savepointTime)) {
try {
client.restoreToSavepoint(savepointTime);
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
return 0;
} else {
} catch (Exception e) {
LOG.info(String.format("The commit \"%s\" failed to roll back.", savepointTime));
return -1;
}

View File

@@ -39,7 +39,7 @@ public class CommitUtil {
public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup) throws IOException {
long totalNew = 0;
HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = target.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
for (String commit : commitsToCatchup) {
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get(),

View File

@@ -30,7 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieCommitArchiveLog;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.junit.After;
import org.junit.Before;
@@ -91,7 +91,7 @@ public class TestArchivedCommitsCommand extends AbstractShellIntegrationTest {
metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
// archive
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
archiveLog.archiveIfRequired(jsc);
}

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.cli.common;
import org.apache.hudi.avro.model.HoodieWriteStat;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.table.HoodieCommitArchiveLog;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import java.util.LinkedHashMap;
import java.util.List;
@@ -36,7 +36,7 @@ public class HoodieTestCommitUtilities {
*/
public static org.apache.hudi.avro.model.HoodieCommitMetadata convertAndOrderCommitMetadata(
HoodieCommitMetadata hoodieCommitMetadata) {
return orderCommitMetadata(HoodieCommitArchiveLog.convertCommitMetadata(hoodieCommitMetadata));
return orderCommitMetadata(HoodieTimelineArchiveLog.convertCommitMetadata(hoodieCommitMetadata));
}
/**