Add CLI support inspect, schedule and run compaction
This commit is contained in:
committed by
vinoth chandar
parent
2e12c86d01
commit
594059a19c
@@ -0,0 +1,229 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.cli.commands;
|
||||
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.cli.HoodiePrintHelper;
|
||||
import com.uber.hoodie.cli.TableHeader;
|
||||
import com.uber.hoodie.cli.commands.SparkMain.SparkCommand;
|
||||
import com.uber.hoodie.cli.utils.InputStreamConsumer;
|
||||
import com.uber.hoodie.cli.utils.SparkUtil;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
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.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.apache.spark.util.Utils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class CompactionCommand implements CommandMarker {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
||||
|
||||
@CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline")
|
||||
public String compactionsAll(
|
||||
@CliOption(key = {
|
||||
"includeExtraMetadata"}, help = "Include extra metadata", unspecifiedDefaultValue = "false") final
|
||||
boolean includeExtraMetadata,
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final
|
||||
boolean headerOnly) throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionTimeline();
|
||||
HoodieTimeline commitTimeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
Set<String> committed = commitTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
|
||||
|
||||
List<HoodieInstant> instants = timeline.getInstants().collect(Collectors.toList());
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
Collections.reverse(instants);
|
||||
for (int i = 0; i < instants.size(); i++) {
|
||||
HoodieInstant instant = instants.get(i);
|
||||
HoodieCompactionPlan workload = null;
|
||||
if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
try {
|
||||
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
|
||||
workload = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
} catch (HoodieIOException ioe) {
|
||||
// SKIP
|
||||
}
|
||||
} else {
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
}
|
||||
|
||||
if (null != workload) {
|
||||
HoodieInstant.State state = instant.getState();
|
||||
if (committed.contains(instant.getTimestamp())) {
|
||||
state = State.COMPLETED;
|
||||
}
|
||||
if (includeExtraMetadata) {
|
||||
rows.add(new Comparable[]{instant.getTimestamp(),
|
||||
state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size(),
|
||||
workload.getExtraMetadata().toString()});
|
||||
} else {
|
||||
rows.add(new Comparable[]{instant.getTimestamp(),
|
||||
state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size()});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Compaction Instant Time")
|
||||
.addTableHeaderField("State")
|
||||
.addTableHeaderField("Total FileIds to be Compacted");
|
||||
if (includeExtraMetadata) {
|
||||
header = header.addTableHeaderField("Extra Metadata");
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant")
|
||||
public String compactionShow(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Base path for the target hoodie dataset") final
|
||||
String compactionInstantTime,
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
if ((null != workload) && (null != workload.getOperations())) {
|
||||
for (HoodieCompactionOperation op : workload.getOperations()) {
|
||||
rows.add(new Comparable[]{op.getPartitionPath(),
|
||||
op.getFileId(),
|
||||
op.getBaseInstantTime(),
|
||||
op.getDataFilePath(),
|
||||
op.getDeltaFilePaths().size(),
|
||||
op.getMetrics().toString()
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant")
|
||||
.addTableHeaderField("Data File Path")
|
||||
.addTableHeaderField("Total Delta Files")
|
||||
.addTableHeaderField("getMetrics");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction schedule", help = "Schedule Compaction")
|
||||
public String scheduleCompact(
|
||||
@CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName,
|
||||
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField,
|
||||
@CliOption(key = {
|
||||
"parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String
|
||||
schemaFilePath,
|
||||
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory,
|
||||
@CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
// First get a compaction instant time and pass it to spark launcher for scheduling compaction
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
tableName, compactionInstantTime, rowKeyField, parallelism, schemaFilePath, sparkMemory, retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to run compaction for " + compactionInstantTime;
|
||||
}
|
||||
return "Compaction successfully completed for " + compactionInstantTime;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction run", help = "Run Compaction for given instant time")
|
||||
public String compact(
|
||||
@CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName,
|
||||
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField,
|
||||
@CliOption(key = {
|
||||
"parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction") final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String
|
||||
schemaFilePath,
|
||||
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory,
|
||||
@CliOption(key = "retry", mandatory = true, 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();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
tableName, compactionInstantTime, rowKeyField, parallelism, schemaFilePath, sparkMemory, retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to run compaction for " + compactionInstantTime;
|
||||
}
|
||||
return "Compaction successfully completed for " + compactionInstantTime;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -29,14 +29,17 @@ import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
@Component
|
||||
public class HDFSParquetImportCommand implements CommandMarker {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
||||
|
||||
@CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset")
|
||||
@CliCommand(value = "hdfsparquetimport", help = "Imports Parquet dataset to a hoodie dataset")
|
||||
public String convert(
|
||||
@CliOption(key = "upsert", mandatory = false, unspecifiedDefaultValue = "false",
|
||||
help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert,
|
||||
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath,
|
||||
@CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") final String srcType,
|
||||
@CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String
|
||||
@@ -59,10 +62,16 @@ public class HDFSParquetImportCommand implements CommandMarker {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
|
||||
sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName, tableType, rowKeyField,
|
||||
String cmd = SparkCommand.IMPORT.toString();
|
||||
if (useUpsert) {
|
||||
cmd = SparkCommand.UPSERT.toString();
|
||||
}
|
||||
|
||||
sparkLauncher.addAppArgs(cmd, srcPath, targetPath, tableName, tableType, rowKeyField,
|
||||
partitionPathField, parallelism, schemaFilePath, sparkMemory, retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
|
||||
@@ -24,6 +24,7 @@ import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.utilities.HDFSParquetImporter;
|
||||
import com.uber.hoodie.utilities.HoodieCompactor;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
@@ -32,12 +33,11 @@ public class SparkMain {
|
||||
|
||||
protected static final Logger LOG = Logger.getLogger(SparkMain.class);
|
||||
|
||||
|
||||
/**
|
||||
* Commands
|
||||
*/
|
||||
enum SparkCommand {
|
||||
ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT
|
||||
ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN,
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
@@ -62,9 +62,20 @@ public class SparkMain {
|
||||
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
|
||||
break;
|
||||
case IMPORT:
|
||||
case UPSERT:
|
||||
assert (args.length == 11);
|
||||
returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6], Integer.parseInt(args[7]),
|
||||
args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10]));
|
||||
returnCode = dataLoad(jsc, command, args[1], args[2], args[3], args[4], args[5], args[6],
|
||||
Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10]));
|
||||
break;
|
||||
case COMPACT_RUN:
|
||||
assert (args.length == 9);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], args[4], args[5], Integer.parseInt(args[6]),
|
||||
args[7], args[8], Integer.parseInt(args[9]), false);
|
||||
break;
|
||||
case COMPACT_SCHEDULE:
|
||||
assert (args.length == 10);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], args[4], args[5], Integer.parseInt(args[6]),
|
||||
args[7], args[8], Integer.parseInt(args[9]), true);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
@@ -73,10 +84,12 @@ public class SparkMain {
|
||||
System.exit(returnCode);
|
||||
}
|
||||
|
||||
private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath, String tableName,
|
||||
private static int dataLoad(JavaSparkContext jsc, String command,
|
||||
String srcPath, String targetPath, String tableName,
|
||||
String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster,
|
||||
String sparkMemory, int retry) throws Exception {
|
||||
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
|
||||
cfg.command = command;
|
||||
cfg.srcPath = srcPath;
|
||||
cfg.targetPath = targetPath;
|
||||
cfg.tableName = tableName;
|
||||
@@ -89,6 +102,22 @@ public class SparkMain {
|
||||
return new HDFSParquetImporter(cfg).dataImport(jsc, retry);
|
||||
}
|
||||
|
||||
private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant,
|
||||
String rowKey, String partitionKey, int parallelism, String schemaFile,
|
||||
String sparkMemory, int retry, boolean schedule) throws Exception {
|
||||
HoodieCompactor.Config cfg = new HoodieCompactor.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.tableName = tableName;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.rowKey = rowKey;
|
||||
cfg.partitionKey = partitionKey;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.schemaFile = schemaFile;
|
||||
cfg.runSchedule = schedule;
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
return new HoodieCompactor(cfg).compact(jsc, retry);
|
||||
}
|
||||
|
||||
private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath,
|
||||
String repairedOutputPath, String basePath) throws Exception {
|
||||
DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc),
|
||||
|
||||
Reference in New Issue
Block a user