tool for importing hive tables (in parquet format) into hoodie dataset (#89)
* tool for importing hive tables (in parquet format) into hoodie dataset * review fixes * review fixes * review fixes
This commit is contained in:
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* Copyright (c) 2017 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.cli.HoodieCLI;
|
||||
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.utilities.HDFSParquetImporter;
|
||||
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 HDFSParquetImportCommand implements CommandMarker {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
||||
|
||||
@CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset")
|
||||
public String convert(
|
||||
@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 targetPath,
|
||||
@CliOption(key = "tableName", mandatory = true, help = "Table name")
|
||||
final String tableName,
|
||||
@CliOption(key = "tableType", mandatory = true, help = "Table type")
|
||||
final String tableType,
|
||||
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name")
|
||||
final String rowKeyField,
|
||||
@CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name")
|
||||
final String partitionPathField,
|
||||
@CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert")
|
||||
final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file")
|
||||
final String schemaFilePath,
|
||||
@CliOption(key = "format", mandatory = true, help = "Format for the input data")
|
||||
final String format,
|
||||
@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 {
|
||||
|
||||
validate(format, srcType);
|
||||
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String sparkPropertiesPath = Utils
|
||||
.getDefaultPropertiesFile(scala.collection.JavaConversions.asScalaMap(System.getenv()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
|
||||
sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName,
|
||||
tableType, rowKeyField, partitionPathField, parallelism, schemaFilePath, sparkMemory,
|
||||
retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to import dataset to hoodie format";
|
||||
}
|
||||
return "Dataset imported to hoodie format";
|
||||
}
|
||||
|
||||
private void validate(String format, String srcType) {
|
||||
(new HDFSParquetImporter.FormatValidator()).validate("format", format);
|
||||
(new HDFSParquetImporter.SourceTypeValidator()).validate("srcType", srcType);
|
||||
}
|
||||
}
|
||||
@@ -1,5 +1,5 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
* Copyright (c) 2016,2017 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.
|
||||
@@ -23,13 +23,11 @@ import com.uber.hoodie.common.util.FSUtils;
|
||||
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 org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
import java.util.Date;
|
||||
|
||||
public class SparkMain {
|
||||
|
||||
protected final static Logger LOG = Logger.getLogger(SparkMain.class);
|
||||
@@ -41,7 +39,9 @@ public class SparkMain {
|
||||
enum SparkCommand {
|
||||
ROLLBACK,
|
||||
DEDUPLICATE,
|
||||
ROLLBACK_TO_SAVEPOINT, SAVEPOINT
|
||||
ROLLBACK_TO_SAVEPOINT,
|
||||
SAVEPOINT,
|
||||
IMPORT
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
@@ -52,20 +52,46 @@ public class SparkMain {
|
||||
|
||||
JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command);
|
||||
int returnCode = 0;
|
||||
if (SparkCommand.ROLLBACK.equals(cmd)) {
|
||||
assert (args.length == 3);
|
||||
returnCode = rollback(jsc, args[1], args[2]);
|
||||
} else if(SparkCommand.DEDUPLICATE.equals(cmd)) {
|
||||
assert (args.length == 4);
|
||||
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
|
||||
} else if(SparkCommand.ROLLBACK_TO_SAVEPOINT.equals(cmd)) {
|
||||
assert (args.length == 3);
|
||||
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
|
||||
switch(cmd) {
|
||||
case ROLLBACK:
|
||||
assert (args.length == 3);
|
||||
returnCode = rollback(jsc, args[1], args[2]);
|
||||
break;
|
||||
case DEDUPLICATE:
|
||||
assert (args.length == 4);
|
||||
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
|
||||
break;
|
||||
case ROLLBACK_TO_SAVEPOINT:
|
||||
assert (args.length == 3);
|
||||
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
|
||||
break;
|
||||
case IMPORT:
|
||||
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]));
|
||||
break;
|
||||
}
|
||||
|
||||
System.exit(returnCode);
|
||||
}
|
||||
|
||||
private static int dataImport(JavaSparkContext jsc, 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.srcPath = srcPath;
|
||||
cfg.targetPath = targetPath;
|
||||
cfg.tableName = tableName;
|
||||
cfg.tableType = tableType;
|
||||
cfg.rowKey = rowKey;
|
||||
cfg.partitionKey = partitionKey;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.schemaFile = schemaFile;
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
return new HDFSParquetImporter(cfg).dataImport(jsc, retry);
|
||||
}
|
||||
|
||||
private static int deduplicatePartitionPath(JavaSparkContext jsc,
|
||||
String duplicatedPartitionPath,
|
||||
String repairedOutputPath,
|
||||
|
||||
@@ -30,6 +30,7 @@ import java.net.URISyntaxException;
|
||||
public class SparkUtil {
|
||||
|
||||
public static Logger logger = Logger.getLogger(SparkUtil.class);
|
||||
public static final String DEFUALT_SPARK_MASTER = "yarn-client";
|
||||
|
||||
/**
|
||||
*
|
||||
@@ -55,7 +56,7 @@ public class SparkUtil {
|
||||
|
||||
public static JavaSparkContext initJavaSparkConf(String name) {
|
||||
SparkConf sparkConf = new SparkConf().setAppName(name);
|
||||
sparkConf.setMaster("yarn-client");
|
||||
sparkConf.setMaster(DEFUALT_SPARK_MASTER);
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||
|
||||
Reference in New Issue
Block a user