Compaction validate, unschedule and repair
This commit is contained in:
committed by
vinoth chandar
parent
d904fe69ca
commit
07324e7a20
@@ -16,6 +16,10 @@
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
import com.uber.hoodie.CompactionAdminClient.RenameOpResult;
|
||||
import com.uber.hoodie.CompactionAdminClient.ValidationOpResult;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
@@ -32,14 +36,20 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
@@ -53,7 +63,9 @@ import org.springframework.stereotype.Component;
|
||||
@Component
|
||||
public class CompactionCommand implements CommandMarker {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
||||
private static Logger log = LogManager.getLogger(CompactionCommand.class);
|
||||
|
||||
private static final String TMP_DIR = "/tmp/";
|
||||
|
||||
@CliAvailabilityIndicator({"compactions show all", "compaction show", "compaction run", "compaction schedule"})
|
||||
public boolean isAvailable() {
|
||||
@@ -84,7 +96,7 @@ public class CompactionCommand implements CommandMarker {
|
||||
for (int i = 0; i < instants.size(); i++) {
|
||||
HoodieInstant instant = instants.get(i);
|
||||
HoodieCompactionPlan workload = null;
|
||||
if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
if (!instant.getAction().equals(COMPACTION_ACTION)) {
|
||||
try {
|
||||
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
|
||||
workload = AvroUtils.deserializeCompactionPlan(
|
||||
@@ -203,7 +215,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
final String schemaFilePath,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G", help = "Spark executor memory")
|
||||
final String sparkMemory,
|
||||
@CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") final String retry,
|
||||
@CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries")
|
||||
final String retry,
|
||||
@CliOption(key = "compactionInstant", mandatory = true, help = "Base path for the target hoodie dataset")
|
||||
final String compactionInstantTime) throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
@@ -227,4 +240,272 @@ public class CompactionCommand implements CommandMarker {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
private static String getTmpSerializerFile() {
|
||||
return TMP_DIR + UUID.randomUUID().toString() + ".ser";
|
||||
}
|
||||
|
||||
private <T> T deSerializeOperationResult(String inputP, FileSystem fs) throws Exception {
|
||||
Path inputPath = new Path(inputP);
|
||||
FSDataInputStream fsDataInputStream = fs.open(inputPath);
|
||||
ObjectInputStream in = new ObjectInputStream(fsDataInputStream);
|
||||
try {
|
||||
T result = (T) in.readObject();
|
||||
log.info("Result : " + result);
|
||||
return result;
|
||||
} finally {
|
||||
in.close();
|
||||
fsDataInputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction validate", help = "Validate Compaction")
|
||||
public String validateCompaction(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
|
||||
@CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = null;
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_VALIDATE.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to validate compaction for " + compactionInstant;
|
||||
}
|
||||
List<ValidationOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
boolean valid = res.stream().map(r -> r.isSuccess()).reduce(Boolean::logicalAnd).orElse(true);
|
||||
String message = "\n\n\t COMPACTION PLAN " + (valid ? "VALID" : "INVALID") + "\n\n";
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[]{r.getOperation().getFileId(),
|
||||
r.getOperation().getBaseInstantTime(),
|
||||
r.getOperation().getDataFilePath().isPresent() ? r.getOperation().getDataFilePath().get() : "",
|
||||
r.getOperation().getDeltaFilePaths().size(), r.isSuccess(),
|
||||
r.getException().isPresent() ? r.getException().get().getMessage() : ""};
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant Time")
|
||||
.addTableHeaderField("Base Data File")
|
||||
.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Valid")
|
||||
.addTableHeaderField("Error");
|
||||
|
||||
output = message + HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit,
|
||||
headerOnly, rows);
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction unschedule", help = "Unschedule Compaction")
|
||||
public String unscheduleCompaction(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
|
||||
@CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV,
|
||||
@CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_PLAN.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to unschedule compaction for " + compactionInstant;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly,
|
||||
"unschedule pending compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction unscheduleFileId", help = "UnSchedule Compaction for a fileId")
|
||||
public String unscheduleCompactFile(
|
||||
@CliOption(key = "fileId", mandatory = true, help = "File Id") final String fileId,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV,
|
||||
@CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_FILE.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), fileId, outputPathStr, "1", master,
|
||||
sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to unschedule compaction for file " + fileId;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly,
|
||||
"unschedule file from pending compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction repair", help = "Renames the files to make them consistent with the timeline as "
|
||||
+ "dictated by Hoodie metadata. Use when compaction unschedule fails partially.")
|
||||
public String repairCompaction(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
|
||||
@CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_REPAIR.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory, Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to unschedule compaction for " + compactionInstant;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly, "repair compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
private String getRenamesToBePrinted(List<RenameOpResult> res, Integer limit,
|
||||
String sortByField, boolean descending, boolean headerOnly, String operation) {
|
||||
|
||||
Optional<Boolean> result = res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd);
|
||||
if (result.isPresent()) {
|
||||
System.out.println("There were some file renames that needed to be done to " + operation);
|
||||
|
||||
if (result.get()) {
|
||||
System.out.println("All renames successfully completed to " + operation + " done !!");
|
||||
} else {
|
||||
System.out.println("Some renames failed. DataSet could be in inconsistent-state. "
|
||||
+ "Try running compaction repair");
|
||||
}
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[] {
|
||||
r.getOperation().fileId, r.getOperation().srcPath, r.getOperation().destPath,
|
||||
r.isExecuted(), r.isSuccess(), r.getException().isPresent() ? r.getException().get().getMessage() : ""
|
||||
};
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Source File Path")
|
||||
.addTableHeaderField("Destination File Path")
|
||||
.addTableHeaderField("Rename Executed?")
|
||||
.addTableHeaderField("Rename Succeeded?")
|
||||
.addTableHeaderField("Error");
|
||||
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending,
|
||||
limit, headerOnly, rows);
|
||||
} else {
|
||||
return "No File renames needed to " + operation + ". Operation successful.";
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -25,6 +25,8 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy;
|
||||
import com.uber.hoodie.utilities.HDFSParquetImporter;
|
||||
import com.uber.hoodie.utilities.HoodieCompactionAdminTool;
|
||||
import com.uber.hoodie.utilities.HoodieCompactionAdminTool.Operation;
|
||||
import com.uber.hoodie.utilities.HoodieCompactor;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -39,6 +41,7 @@ public class SparkMain {
|
||||
*/
|
||||
enum SparkCommand {
|
||||
ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN,
|
||||
COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
@@ -78,10 +81,32 @@ public class SparkMain {
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], 1,
|
||||
"", args[4], 0, true);
|
||||
break;
|
||||
case COMPACT_VALIDATE:
|
||||
assert (args.length == 7);
|
||||
doCompactValidate(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6]);
|
||||
returnCode = 0;
|
||||
break;
|
||||
case COMPACT_REPAIR:
|
||||
assert (args.length == 8);
|
||||
doCompactRepair(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Boolean.valueOf(args[7]));
|
||||
returnCode = 0;
|
||||
break;
|
||||
case COMPACT_UNSCHEDULE_FILE:
|
||||
assert (args.length == 9);
|
||||
doCompactUnscheduleFile(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Boolean.valueOf(args[7]), Boolean.valueOf(args[8]));
|
||||
returnCode = 0;
|
||||
break;
|
||||
case COMPACT_UNSCHEDULE_PLAN:
|
||||
assert (args.length == 9);
|
||||
doCompactUnschedule(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Boolean.valueOf(args[7]), Boolean.valueOf(args[8]));
|
||||
returnCode = 0;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
System.exit(returnCode);
|
||||
}
|
||||
|
||||
@@ -103,6 +128,73 @@ public class SparkMain {
|
||||
return new HDFSParquetImporter(cfg).dataImport(jsc, retry);
|
||||
}
|
||||
|
||||
private static void doCompactValidate(JavaSparkContext jsc, String basePath, String compactionInstant,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.VALIDATE;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.parallelism = parallelism;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactRepair(JavaSparkContext jsc, String basePath, String compactionInstant,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean dryRun) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.REPAIR;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.dryRun = dryRun;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactUnschedule(JavaSparkContext jsc, String basePath, String compactionInstant,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation,
|
||||
boolean dryRun) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.UNSCHEDULE_PLAN;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.dryRun = dryRun;
|
||||
cfg.skipValidation = skipValidation;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation,
|
||||
boolean dryRun) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.UNSCHEDULE_FILE;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.fileId = fileId;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.dryRun = dryRun;
|
||||
cfg.skipValidation = skipValidation;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant,
|
||||
int parallelism, String schemaFile, String sparkMemory, int retry, boolean schedule) throws Exception {
|
||||
HoodieCompactor.Config cfg = new HoodieCompactor.Config();
|
||||
|
||||
Reference in New Issue
Block a user