Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
@@ -60,253 +60,255 @@ import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.VoidFunction;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class HDFSParquetImporter implements Serializable{
|
||||
public class HDFSParquetImporter implements Serializable {
|
||||
|
||||
private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class);
|
||||
private final Config cfg;
|
||||
private final transient FileSystem fs;
|
||||
public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd");
|
||||
private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class);
|
||||
private final Config cfg;
|
||||
private final transient FileSystem fs;
|
||||
public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd");
|
||||
|
||||
public HDFSParquetImporter(
|
||||
Config cfg) throws IOException {
|
||||
this.cfg = cfg;
|
||||
fs = FSUtils.getFs();
|
||||
public HDFSParquetImporter(
|
||||
Config cfg) throws IOException {
|
||||
this.cfg = cfg;
|
||||
fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
public static class FormatValidator implements IValueValidator<String> {
|
||||
|
||||
List<String> validFormats = Arrays.asList("parquet");
|
||||
|
||||
@Override
|
||||
public void validate(String name, String value) throws ParameterException {
|
||||
if (value == null || !validFormats.contains(value)) {
|
||||
throw new ParameterException(String
|
||||
.format("Invalid format type: value:%s: supported formats:%s", value,
|
||||
validFormats));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class SourceTypeValidator implements IValueValidator<String> {
|
||||
|
||||
List<String> validSourceTypes = Arrays.asList("hdfs");
|
||||
|
||||
@Override
|
||||
public void validate(String name, String value) throws ParameterException {
|
||||
if (value == null || !validSourceTypes.contains(value)) {
|
||||
throw new ParameterException(String
|
||||
.format("Invalid source type: value:%s: supported source types:%s", value,
|
||||
validSourceTypes));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--src-path",
|
||||
"-sp"}, description = "Base path for the input dataset", required = true)
|
||||
public String srcPath = null;
|
||||
@Parameter(names = {"--src-type",
|
||||
"-st"}, description = "Source type for the input dataset", required = true,
|
||||
validateValueWith = SourceTypeValidator.class)
|
||||
public String srcType = null;
|
||||
@Parameter(names = {"--target-path",
|
||||
"-tp"}, description = "Base path for the target hoodie dataset", required = true)
|
||||
public String targetPath = null;
|
||||
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
|
||||
public String tableName = null;
|
||||
@Parameter(names = {"--table-type", "-tt"}, description = "Table type", required = true)
|
||||
public String tableType = null;
|
||||
@Parameter(names = {"--row-key-field",
|
||||
"-rk"}, description = "Row key field name", required = true)
|
||||
public String rowKey = null;
|
||||
@Parameter(names = {"--partition-key-field",
|
||||
"-pk"}, description = "Partition key field name", required = true)
|
||||
public String partitionKey = null;
|
||||
@Parameter(names = {"--parallelism",
|
||||
"-pl"}, description = "Parallelism for hoodie insert", required = true)
|
||||
public int parallelism = 1;
|
||||
@Parameter(names = {"--schema-file",
|
||||
"-sf"}, description = "path for Avro schema file", required = true)
|
||||
public String schemaFile = null;
|
||||
@Parameter(names = {"--format",
|
||||
"-f"}, description = "Format for the input data.", required = false,
|
||||
validateValueWith = FormatValidator.class)
|
||||
public String format = null;
|
||||
@Parameter(names = {"--spark-master",
|
||||
"-ms"}, description = "Spark master", required = false)
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory",
|
||||
"-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--retry",
|
||||
"-rt"}, description = "number of retries", required = false)
|
||||
public int retry = 0;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public static void main(String args[]) throws Exception {
|
||||
final HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg);
|
||||
dataImporter.dataImport(dataImporter.getSparkContext(), cfg.retry);
|
||||
}
|
||||
|
||||
private JavaSparkContext getSparkContext() {
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-data-importer-" + cfg.tableName);
|
||||
sparkConf.setMaster(cfg.sparkMaster);
|
||||
|
||||
if (cfg.sparkMaster.startsWith("yarn")) {
|
||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||
sparkConf.set("spark.eventLog.enabled", "true");
|
||||
}
|
||||
|
||||
public static class FormatValidator implements IValueValidator<String> {
|
||||
List<String> validFormats = Arrays.asList("parquet");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.executor.memory", cfg.sparkMemory);
|
||||
|
||||
// Configure hadoop conf
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec",
|
||||
"org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
return new JavaSparkContext(sparkConf);
|
||||
}
|
||||
|
||||
private String getSchema() throws Exception {
|
||||
// Read schema file.
|
||||
Path p = new Path(cfg.schemaFile);
|
||||
if (!fs.exists(p)) {
|
||||
throw new Exception(
|
||||
String.format("Could not find - %s - schema file.", cfg.schemaFile));
|
||||
}
|
||||
long len = fs.getFileStatus(p).getLen();
|
||||
ByteBuffer buf = ByteBuffer.allocate((int) len);
|
||||
FSDataInputStream inputStream = null;
|
||||
try {
|
||||
inputStream = fs.open(p);
|
||||
inputStream.readFully(0, buf.array(), 0, buf.array().length);
|
||||
} finally {
|
||||
if (inputStream != null) {
|
||||
inputStream.close();
|
||||
}
|
||||
}
|
||||
return new String(buf.array());
|
||||
}
|
||||
|
||||
public int dataImport(JavaSparkContext jsc, int retry) throws Exception {
|
||||
int ret = -1;
|
||||
try {
|
||||
// Verify that targetPath is not present.
|
||||
if (fs.exists(new Path(cfg.targetPath))) {
|
||||
throw new HoodieIOException(
|
||||
String.format("Make sure %s is not present.", cfg.targetPath));
|
||||
}
|
||||
do {
|
||||
ret = dataImport(jsc);
|
||||
} while (ret != 0 && retry-- > 0);
|
||||
} catch (Throwable t) {
|
||||
logger.error(t);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected int dataImport(JavaSparkContext jsc) throws IOException {
|
||||
try {
|
||||
if (fs.exists(new Path(cfg.targetPath))) {
|
||||
// cleanup target directory.
|
||||
fs.delete(new Path(cfg.targetPath), true);
|
||||
}
|
||||
|
||||
//Get schema.
|
||||
String schemaStr = getSchema();
|
||||
|
||||
// Initialize target hoodie table.
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName);
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType);
|
||||
HoodieTableMetaClient.initializePathAsHoodieDataset(fs, cfg.targetPath, properties);
|
||||
|
||||
HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr,
|
||||
cfg.parallelism);
|
||||
|
||||
Job job = Job.getInstance(jsc.hadoopConfiguration());
|
||||
// To parallelize reading file status.
|
||||
job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024");
|
||||
AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(),
|
||||
(new Schema.Parser().parse(schemaStr)));
|
||||
ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
|
||||
|
||||
JavaRDD<HoodieRecord<HoodieJsonPayload>> hoodieRecords = jsc
|
||||
.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class,
|
||||
GenericRecord.class, job.getConfiguration())
|
||||
// To reduce large number of tasks.
|
||||
.coalesce(16 * cfg.parallelism)
|
||||
.map(new Function<Tuple2<Void, GenericRecord>, HoodieRecord<HoodieJsonPayload>>() {
|
||||
@Override
|
||||
public HoodieRecord<HoodieJsonPayload> call(Tuple2<Void, GenericRecord> entry)
|
||||
throws Exception {
|
||||
GenericRecord genericRecord = entry._2();
|
||||
Object partitionField = genericRecord.get(cfg.partitionKey);
|
||||
if (partitionField == null) {
|
||||
throw new HoodieIOException(
|
||||
"partition key is missing. :" + cfg.partitionKey);
|
||||
}
|
||||
Object rowField = genericRecord.get(cfg.rowKey);
|
||||
if (rowField == null) {
|
||||
throw new HoodieIOException(
|
||||
"row field is missing. :" + cfg.rowKey);
|
||||
}
|
||||
long ts = (long) ((Double) partitionField * 1000l);
|
||||
String partitionPath = PARTITION_FORMATTER.format(new Date(ts));
|
||||
return new HoodieRecord<HoodieJsonPayload>(
|
||||
new HoodieKey((String) rowField, partitionPath),
|
||||
new HoodieJsonPayload(genericRecord.toString()));
|
||||
}
|
||||
}
|
||||
);
|
||||
// Get commit time.
|
||||
String commitTime = client.startCommit();
|
||||
|
||||
JavaRDD<WriteStatus> writeResponse = client.bulkInsert(hoodieRecords, commitTime);
|
||||
Accumulator<Integer> errors = jsc.accumulator(0);
|
||||
writeResponse.foreach(new VoidFunction<WriteStatus>() {
|
||||
@Override
|
||||
public void validate(String name, String value) throws ParameterException {
|
||||
if (value == null || !validFormats.contains(value)) {
|
||||
throw new ParameterException(String
|
||||
.format("Invalid format type: value:%s: supported formats:%s", value,
|
||||
validFormats));
|
||||
}
|
||||
public void call(WriteStatus writeStatus) throws Exception {
|
||||
if (writeStatus.hasErrors()) {
|
||||
errors.add(1);
|
||||
logger.error(String.format("Error processing records :writeStatus:%s",
|
||||
writeStatus.getStat().toString()));
|
||||
}
|
||||
}
|
||||
});
|
||||
if (errors.value() == 0) {
|
||||
logger.info(String
|
||||
.format("Dataset imported into hoodie dataset with %s commit time.",
|
||||
commitTime));
|
||||
return 0;
|
||||
}
|
||||
logger.error(String.format("Import failed with %d errors.", errors.value()));
|
||||
} catch (Throwable t) {
|
||||
logger.error("Error occurred.", t);
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
public static class SourceTypeValidator implements IValueValidator<String> {
|
||||
List<String> validSourceTypes = Arrays.asList("hdfs");
|
||||
|
||||
@Override
|
||||
public void validate(String name, String value) throws ParameterException {
|
||||
if (value == null || !validSourceTypes.contains(value)) {
|
||||
throw new ParameterException(String
|
||||
.format("Invalid source type: value:%s: supported source types:%s", value,
|
||||
validSourceTypes));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--src-path",
|
||||
"-sp"}, description = "Base path for the input dataset", required = true)
|
||||
public String srcPath = null;
|
||||
@Parameter(names = {"--src-type",
|
||||
"-st"}, description = "Source type for the input dataset", required = true,
|
||||
validateValueWith = SourceTypeValidator.class)
|
||||
public String srcType = null;
|
||||
@Parameter(names = {"--target-path",
|
||||
"-tp"}, description = "Base path for the target hoodie dataset", required = true)
|
||||
public String targetPath = null;
|
||||
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
|
||||
public String tableName = null;
|
||||
@Parameter(names = {"--table-type", "-tt"}, description = "Table type", required = true)
|
||||
public String tableType = null;
|
||||
@Parameter(names = {"--row-key-field",
|
||||
"-rk"}, description = "Row key field name", required = true)
|
||||
public String rowKey = null;
|
||||
@Parameter(names = {"--partition-key-field",
|
||||
"-pk"}, description = "Partition key field name", required = true)
|
||||
public String partitionKey = null;
|
||||
@Parameter(names = {"--parallelism",
|
||||
"-pl"}, description = "Parallelism for hoodie insert", required = true)
|
||||
public int parallelism = 1;
|
||||
@Parameter(names = {"--schema-file",
|
||||
"-sf"}, description = "path for Avro schema file", required = true)
|
||||
public String schemaFile = null;
|
||||
@Parameter(names = {"--format",
|
||||
"-f"}, description = "Format for the input data.", required = false,
|
||||
validateValueWith = FormatValidator.class)
|
||||
public String format = null;
|
||||
@Parameter(names = {"--spark-master",
|
||||
"-ms"}, description = "Spark master", required = false)
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory",
|
||||
"-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--retry",
|
||||
"-rt"}, description = "number of retries", required = false)
|
||||
public int retry = 0;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public static void main(String args[]) throws Exception {
|
||||
final HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg);
|
||||
dataImporter.dataImport(dataImporter.getSparkContext(), cfg.retry);
|
||||
}
|
||||
|
||||
private JavaSparkContext getSparkContext() {
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-data-importer-" + cfg.tableName);
|
||||
sparkConf.setMaster(cfg.sparkMaster);
|
||||
|
||||
if (cfg.sparkMaster.startsWith("yarn")) {
|
||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||
sparkConf.set("spark.eventLog.enabled", "true");
|
||||
}
|
||||
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.executor.memory", cfg.sparkMemory);
|
||||
|
||||
// Configure hadoop conf
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec",
|
||||
"org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
return new JavaSparkContext(sparkConf);
|
||||
}
|
||||
|
||||
private String getSchema() throws Exception {
|
||||
// Read schema file.
|
||||
Path p = new Path(cfg.schemaFile);
|
||||
if (!fs.exists(p)) {
|
||||
throw new Exception(
|
||||
String.format("Could not find - %s - schema file.", cfg.schemaFile));
|
||||
}
|
||||
long len = fs.getFileStatus(p).getLen();
|
||||
ByteBuffer buf = ByteBuffer.allocate((int) len);
|
||||
FSDataInputStream inputStream = null;
|
||||
try {
|
||||
inputStream = fs.open(p);
|
||||
inputStream.readFully(0, buf.array(), 0, buf.array().length);
|
||||
}
|
||||
finally {
|
||||
if (inputStream != null)
|
||||
inputStream.close();
|
||||
}
|
||||
return new String(buf.array());
|
||||
}
|
||||
|
||||
public int dataImport(JavaSparkContext jsc, int retry) throws Exception {
|
||||
int ret = -1;
|
||||
try {
|
||||
// Verify that targetPath is not present.
|
||||
if (fs.exists(new Path(cfg.targetPath))) {
|
||||
throw new HoodieIOException(
|
||||
String.format("Make sure %s is not present.", cfg.targetPath));
|
||||
}
|
||||
do {
|
||||
ret = dataImport(jsc);
|
||||
} while (ret != 0 && retry-- > 0);
|
||||
} catch (Throwable t) {
|
||||
logger.error(t);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected int dataImport(JavaSparkContext jsc) throws IOException {
|
||||
try {
|
||||
if (fs.exists(new Path(cfg.targetPath))) {
|
||||
// cleanup target directory.
|
||||
fs.delete(new Path(cfg.targetPath), true);
|
||||
}
|
||||
|
||||
//Get schema.
|
||||
String schemaStr = getSchema();
|
||||
|
||||
// Initialize target hoodie table.
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName);
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType);
|
||||
HoodieTableMetaClient.initializePathAsHoodieDataset(fs, cfg.targetPath, properties);
|
||||
|
||||
HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr,
|
||||
cfg.parallelism);
|
||||
|
||||
Job job = Job.getInstance(jsc.hadoopConfiguration());
|
||||
// To parallelize reading file status.
|
||||
job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024");
|
||||
AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(),
|
||||
(new Schema.Parser().parse(schemaStr)));
|
||||
ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
|
||||
|
||||
JavaRDD<HoodieRecord<HoodieJsonPayload>> hoodieRecords = jsc
|
||||
.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class,
|
||||
GenericRecord.class, job.getConfiguration())
|
||||
// To reduce large number of tasks.
|
||||
.coalesce(16 * cfg.parallelism)
|
||||
.map(new Function<Tuple2<Void, GenericRecord>, HoodieRecord<HoodieJsonPayload>>() {
|
||||
@Override
|
||||
public HoodieRecord<HoodieJsonPayload> call(Tuple2<Void, GenericRecord> entry)
|
||||
throws Exception {
|
||||
GenericRecord genericRecord = entry._2();
|
||||
Object partitionField = genericRecord.get(cfg.partitionKey);
|
||||
if (partitionField == null) {
|
||||
throw new HoodieIOException(
|
||||
"partition key is missing. :" + cfg.partitionKey);
|
||||
}
|
||||
Object rowField = genericRecord.get(cfg.rowKey);
|
||||
if (rowField == null) {
|
||||
throw new HoodieIOException(
|
||||
"row field is missing. :" + cfg.rowKey);
|
||||
}
|
||||
long ts = (long) ((Double) partitionField * 1000l);
|
||||
String partitionPath = PARTITION_FORMATTER.format(new Date(ts));
|
||||
return new HoodieRecord<HoodieJsonPayload>(
|
||||
new HoodieKey((String) rowField, partitionPath),
|
||||
new HoodieJsonPayload(genericRecord.toString()));
|
||||
}
|
||||
}
|
||||
);
|
||||
// Get commit time.
|
||||
String commitTime = client.startCommit();
|
||||
|
||||
JavaRDD<WriteStatus> writeResponse = client.bulkInsert(hoodieRecords, commitTime);
|
||||
Accumulator<Integer> errors = jsc.accumulator(0);
|
||||
writeResponse.foreach(new VoidFunction<WriteStatus>() {
|
||||
@Override
|
||||
public void call(WriteStatus writeStatus) throws Exception {
|
||||
if (writeStatus.hasErrors()) {
|
||||
errors.add(1);
|
||||
logger.error(String.format("Error processing records :writeStatus:%s",
|
||||
writeStatus.getStat().toString()));
|
||||
}
|
||||
}
|
||||
});
|
||||
if (errors.value() == 0) {
|
||||
logger.info(String
|
||||
.format("Dataset imported into hoodie dataset with %s commit time.",
|
||||
commitTime));
|
||||
return 0;
|
||||
}
|
||||
logger.error(String.format("Import failed with %d errors.", errors.value()));
|
||||
} catch (Throwable t) {
|
||||
logger.error("Error occurred.", t);
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath,
|
||||
String schemaStr, int parallelism) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withParallelism(parallelism, parallelism).withSchema(schemaStr)
|
||||
.combineInput(true, true).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
return new HoodieWriteClient(jsc, config);
|
||||
}
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath,
|
||||
String schemaStr, int parallelism) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withParallelism(parallelism, parallelism).withSchema(schemaStr)
|
||||
.combineInput(true, true).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
return new HoodieWriteClient(jsc, config);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,19 +25,6 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullSQLException;
|
||||
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.stringtemplate.v4.ST;
|
||||
|
||||
import javax.sql.DataSource;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
@@ -50,301 +37,343 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Scanner;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.sql.DataSource;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.stringtemplate.v4.ST;
|
||||
|
||||
/**
|
||||
* Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as another hive temporary table.
|
||||
* Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as
|
||||
* another hive temporary table.
|
||||
*
|
||||
* Current Limitations:
|
||||
*
|
||||
* - Only the source table can be incrementally pulled (usually the largest table)
|
||||
* - The incrementally pulled table can't be referenced more than once.
|
||||
* - Only the source table can be incrementally pulled (usually the largest table) - The
|
||||
* incrementally pulled table can't be referenced more than once.
|
||||
*/
|
||||
public class HiveIncrementalPuller {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HiveIncrementalPuller.class);
|
||||
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||
private static Logger log = LogManager.getLogger(HiveIncrementalPuller.class);
|
||||
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||
|
||||
public static class Config implements Serializable {
|
||||
@Parameter(names = {"--hiveUrl"}) public String hiveJDBCUrl =
|
||||
"jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2";
|
||||
@Parameter(names = {"--hiveUser"}) public String hiveUsername = "hive";
|
||||
@Parameter(names = {"--hivePass"}) public String hivePassword = "";
|
||||
@Parameter(names = {"--queue"}) public String yarnQueueName = "hadoop-queue";
|
||||
@Parameter(names = {"--tmp"}) public String hoodieTmpDir = "/app/hoodie/intermediate";
|
||||
@Parameter(names = {"--extractSQLFile"}, required = true) public String incrementalSQLFile;
|
||||
@Parameter(names = {"--sourceDb"}, required = true) public String sourceDb;
|
||||
@Parameter(names = {"--sourceTable"}, required = true) public String sourceTable;
|
||||
@Parameter(names = {"--targetDb"}) public String targetDb;
|
||||
@Parameter(names = {"--targetTable"}, required = true) public String targetTable;
|
||||
@Parameter(names = {"--tmpdb"}) public String tmpDb = "tmp";
|
||||
@Parameter(names = {"--fromCommitTime"}) public String fromCommitTime;
|
||||
@Parameter(names = {"--maxCommits"}) public int maxCommits = 3;
|
||||
@Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false;
|
||||
@Parameter(names = {"--storageFormat"}) public String tempTableStorageFormat = "AVRO";
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--hiveUrl"})
|
||||
public String hiveJDBCUrl =
|
||||
"jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2";
|
||||
@Parameter(names = {"--hiveUser"})
|
||||
public String hiveUsername = "hive";
|
||||
@Parameter(names = {"--hivePass"})
|
||||
public String hivePassword = "";
|
||||
@Parameter(names = {"--queue"})
|
||||
public String yarnQueueName = "hadoop-queue";
|
||||
@Parameter(names = {"--tmp"})
|
||||
public String hoodieTmpDir = "/app/hoodie/intermediate";
|
||||
@Parameter(names = {"--extractSQLFile"}, required = true)
|
||||
public String incrementalSQLFile;
|
||||
@Parameter(names = {"--sourceDb"}, required = true)
|
||||
public String sourceDb;
|
||||
@Parameter(names = {"--sourceTable"}, required = true)
|
||||
public String sourceTable;
|
||||
@Parameter(names = {"--targetDb"})
|
||||
public String targetDb;
|
||||
@Parameter(names = {"--targetTable"}, required = true)
|
||||
public String targetTable;
|
||||
@Parameter(names = {"--tmpdb"})
|
||||
public String tmpDb = "tmp";
|
||||
@Parameter(names = {"--fromCommitTime"})
|
||||
public String fromCommitTime;
|
||||
@Parameter(names = {"--maxCommits"})
|
||||
public int maxCommits = 3;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
@Parameter(names = {"--storageFormat"})
|
||||
public String tempTableStorageFormat = "AVRO";
|
||||
}
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
}
|
||||
}
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
private Connection connection;
|
||||
protected final Config config;
|
||||
private final ST incrementalPullSQLtemplate;
|
||||
|
||||
public HiveIncrementalPuller(Config config) throws IOException {
|
||||
this.config = config;
|
||||
validateConfig(config);
|
||||
String templateContent = IOUtils
|
||||
.toString(this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
|
||||
incrementalPullSQLtemplate = new ST(templateContent);
|
||||
}
|
||||
|
||||
private void validateConfig(Config config) {
|
||||
if (config.maxCommits == -1) {
|
||||
config.maxCommits = Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
|
||||
public void saveDelta() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Statement stmt = null;
|
||||
try {
|
||||
if (config.fromCommitTime == null) {
|
||||
config.fromCommitTime = inferCommitTime(fs);
|
||||
log.info("FromCommitTime inferred as " + config.fromCommitTime);
|
||||
}
|
||||
|
||||
log.info("FromCommitTime - " + config.fromCommitTime);
|
||||
String sourceTableLocation = getTableLocation(config.sourceDb, config.sourceTable);
|
||||
String lastCommitTime = getLastCommitTimePulled(fs, sourceTableLocation);
|
||||
if (lastCommitTime == null) {
|
||||
log.info("Nothing to pull. However we will continue to create a empty table");
|
||||
lastCommitTime = config.fromCommitTime;
|
||||
}
|
||||
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
// drop the temp table if exists
|
||||
String tempDbTable = config.tmpDb + "." + config.targetTable + "__" + config.sourceTable;
|
||||
String tempDbTablePath =
|
||||
config.hoodieTmpDir + "/" + config.targetTable + "__" + config.sourceTable + "/"
|
||||
+ lastCommitTime;
|
||||
executeStatement("drop table " + tempDbTable, stmt);
|
||||
deleteHDFSPath(fs, tempDbTablePath);
|
||||
if (!ensureTempPathExists(fs, lastCommitTime)) {
|
||||
throw new IllegalStateException(
|
||||
"Could not create target path at " + new Path(config.hoodieTmpDir,
|
||||
config.targetTable + "/" + lastCommitTime));
|
||||
}
|
||||
|
||||
initHiveBeelineProperties(stmt);
|
||||
executeIncrementalSQL(tempDbTable, tempDbTablePath, stmt);
|
||||
log.info("Finished HoodieReader execution");
|
||||
} catch (SQLException e) {
|
||||
log.error("Exception when executing SQL", e);
|
||||
throw new IOException("Could not scan " + config.sourceTable + " incrementally", e);
|
||||
} finally {
|
||||
try {
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
log.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, Statement stmt)
|
||||
throws FileNotFoundException, SQLException {
|
||||
incrementalPullSQLtemplate.add("tempDbTable", tempDbTable);
|
||||
incrementalPullSQLtemplate.add("tempDbTablePath", tempDbTablePath);
|
||||
|
||||
String storedAsClause = getStoredAsClause();
|
||||
|
||||
incrementalPullSQLtemplate.add("storedAsClause", storedAsClause);
|
||||
String incrementalSQL =
|
||||
new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z").next();
|
||||
if (!incrementalSQL.contains(config.sourceDb + "." + config.sourceTable)) {
|
||||
log.info("Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable
|
||||
+ ", which means its pulling from a different table. Fencing this from happening.");
|
||||
throw new HoodieIncrementalPullSQLException(
|
||||
"Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable);
|
||||
}
|
||||
if (!incrementalSQL.contains("`_hoodie_commit_time` > '%targetBasePath'")) {
|
||||
log.info("Incremental SQL : " + incrementalSQL
|
||||
+ " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add this clause for incremental to work properly.");
|
||||
throw new HoodieIncrementalPullSQLException(
|
||||
"Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which means its not pulling incrementally");
|
||||
}
|
||||
|
||||
private Connection connection;
|
||||
protected final Config config;
|
||||
private final ST incrementalPullSQLtemplate;
|
||||
incrementalPullSQLtemplate
|
||||
.add("incrementalSQL", String.format(incrementalSQL, config.fromCommitTime));
|
||||
String sql = incrementalPullSQLtemplate.render();
|
||||
// Check if the SQL is pulling from the right database
|
||||
executeStatement(sql, stmt);
|
||||
}
|
||||
|
||||
public HiveIncrementalPuller(Config config) throws IOException {
|
||||
this.config = config;
|
||||
validateConfig(config);
|
||||
String templateContent = IOUtils.toString(this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
|
||||
incrementalPullSQLtemplate = new ST(templateContent);
|
||||
private String getStoredAsClause() {
|
||||
if (config.tempTableStorageFormat.equalsIgnoreCase("json")) {
|
||||
// Special case for json
|
||||
// default json serde does not support having same key even if its under multiple depths
|
||||
return "ROW FORMAT SERDE 'org.openx.data.jsonserde.JsonSerDe' STORED AS TEXTFILE";
|
||||
}
|
||||
return "STORED AS " + config.tempTableStorageFormat;
|
||||
}
|
||||
|
||||
private void validateConfig(Config config) {
|
||||
if(config.maxCommits == -1) {
|
||||
config.maxCommits = Integer.MAX_VALUE;
|
||||
private void initHiveBeelineProperties(Statement stmt) throws SQLException {
|
||||
log.info("Setting up Hive JDBC Session with properties");
|
||||
// set the queue
|
||||
executeStatement("set mapred.job.queue.name=" + config.yarnQueueName, stmt);
|
||||
// Set the inputformat to HoodieCombineHiveInputFormat
|
||||
executeStatement(
|
||||
"set hive.input.format=com.uber.hoodie.hadoop.hive.HoodieCombineHiveInputFormat", stmt);
|
||||
// Allow queries without partition predicate
|
||||
executeStatement("set hive.strict.checks.large.query=false", stmt);
|
||||
// Dont gather stats for the table created
|
||||
executeStatement("set hive.stats.autogather=false", stmt);
|
||||
// Set the hoodie modie
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.mode=INCREMENTAL", stmt);
|
||||
// Set the from commit time
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.start.timestamp="
|
||||
+ config.fromCommitTime, stmt);
|
||||
// Set number of commits to pull
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String
|
||||
.valueOf(config.maxCommits), stmt);
|
||||
}
|
||||
|
||||
private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException {
|
||||
log.info("Deleting path " + path);
|
||||
return fs.delete(new Path(path), true);
|
||||
}
|
||||
|
||||
private void executeStatement(String sql, Statement stmt) throws SQLException {
|
||||
log.info("Executing: " + sql);
|
||||
stmt.execute(sql);
|
||||
}
|
||||
|
||||
private String inferCommitTime(FileSystem fs) throws SQLException, IOException {
|
||||
log.info("FromCommitTime not specified. Trying to infer it from Hoodie dataset "
|
||||
+ config.targetDb + "." + config.targetTable);
|
||||
String targetDataLocation = getTableLocation(config.targetDb, config.targetTable);
|
||||
return scanForCommitTime(fs, targetDataLocation);
|
||||
}
|
||||
|
||||
private String getTableLocation(String db, String table) throws SQLException {
|
||||
ResultSet resultSet = null;
|
||||
Statement stmt = null;
|
||||
try {
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
resultSet = stmt.executeQuery("describe formatted `" + db + "." + table + "`");
|
||||
while (resultSet.next()) {
|
||||
if (resultSet.getString(1).trim().equals("Location:")) {
|
||||
log.info("Inferred table location for " + db + "." + table + " as " + resultSet
|
||||
.getString(2));
|
||||
return resultSet.getString(2);
|
||||
}
|
||||
}
|
||||
|
||||
public void saveDelta() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Statement stmt = null;
|
||||
try {
|
||||
if (config.fromCommitTime == null) {
|
||||
config.fromCommitTime = inferCommitTime(fs);
|
||||
log.info("FromCommitTime inferred as " + config.fromCommitTime);
|
||||
}
|
||||
|
||||
log.info("FromCommitTime - " + config.fromCommitTime);
|
||||
String sourceTableLocation = getTableLocation(config.sourceDb, config.sourceTable);
|
||||
String lastCommitTime = getLastCommitTimePulled(fs, sourceTableLocation);
|
||||
if (lastCommitTime == null) {
|
||||
log.info("Nothing to pull. However we will continue to create a empty table");
|
||||
lastCommitTime = config.fromCommitTime;
|
||||
}
|
||||
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
// drop the temp table if exists
|
||||
String tempDbTable = config.tmpDb + "." + config.targetTable + "__" + config.sourceTable;
|
||||
String tempDbTablePath = config.hoodieTmpDir + "/" + config.targetTable + "__" + config.sourceTable + "/" + lastCommitTime;
|
||||
executeStatement("drop table " + tempDbTable, stmt);
|
||||
deleteHDFSPath(fs, tempDbTablePath);
|
||||
if (!ensureTempPathExists(fs, lastCommitTime)) {
|
||||
throw new IllegalStateException(
|
||||
"Could not create target path at " + new Path(config.hoodieTmpDir,
|
||||
config.targetTable + "/" + lastCommitTime));
|
||||
}
|
||||
|
||||
initHiveBeelineProperties(stmt);
|
||||
executeIncrementalSQL(tempDbTable, tempDbTablePath, stmt);
|
||||
log.info("Finished HoodieReader execution");
|
||||
} catch (SQLException e) {
|
||||
log.error("Exception when executing SQL", e);
|
||||
throw new IOException("Could not scan " + config.sourceTable + " incrementally", e);
|
||||
} finally {
|
||||
try {
|
||||
if (stmt != null)
|
||||
stmt.close();
|
||||
} catch (SQLException e) {
|
||||
log.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieIncrementalPullException(
|
||||
"Failed to get data location for table " + db + "." + table, e);
|
||||
} finally {
|
||||
try {
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, Statement stmt)
|
||||
throws FileNotFoundException, SQLException {
|
||||
incrementalPullSQLtemplate.add("tempDbTable", tempDbTable);
|
||||
incrementalPullSQLtemplate.add("tempDbTablePath", tempDbTablePath);
|
||||
|
||||
String storedAsClause = getStoredAsClause();
|
||||
|
||||
incrementalPullSQLtemplate.add("storedAsClause", storedAsClause);
|
||||
String incrementalSQL =
|
||||
new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z").next();
|
||||
if (!incrementalSQL.contains(config.sourceDb + "." + config.sourceTable)) {
|
||||
log.info("Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable
|
||||
+ ", which means its pulling from a different table. Fencing this from happening.");
|
||||
throw new HoodieIncrementalPullSQLException(
|
||||
"Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable);
|
||||
}
|
||||
if (!incrementalSQL.contains("`_hoodie_commit_time` > '%targetBasePath'")) {
|
||||
log.info("Incremental SQL : " + incrementalSQL
|
||||
+ " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add this clause for incremental to work properly.");
|
||||
throw new HoodieIncrementalPullSQLException(
|
||||
"Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which means its not pulling incrementally");
|
||||
if (resultSet != null) {
|
||||
resultSet.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
log.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
incrementalPullSQLtemplate
|
||||
.add("incrementalSQL", String.format(incrementalSQL, config.fromCommitTime));
|
||||
String sql = incrementalPullSQLtemplate.render();
|
||||
// Check if the SQL is pulling from the right database
|
||||
executeStatement(sql, stmt);
|
||||
private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IOException {
|
||||
if (targetDataPath == null) {
|
||||
throw new IllegalArgumentException(
|
||||
"Please specify either --fromCommitTime or --targetDataPath");
|
||||
}
|
||||
if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
|
||||
return "0";
|
||||
}
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
|
||||
|
||||
Optional<HoodieInstant>
|
||||
lastCommit = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
if (lastCommit.isPresent()) {
|
||||
return lastCommit.get().getTimestamp();
|
||||
}
|
||||
return "0";
|
||||
}
|
||||
|
||||
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
|
||||
throws IOException {
|
||||
Path targetBaseDirPath = new Path(config.hoodieTmpDir,
|
||||
config.targetTable + "__" + config.sourceTable);
|
||||
if (!fs.exists(targetBaseDirPath)) {
|
||||
log.info("Creating " + targetBaseDirPath + " with permission drwxrwxrwx");
|
||||
boolean result = FileSystem.mkdirs(fs, targetBaseDirPath,
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
if (!result) {
|
||||
throw new HoodieException(
|
||||
"Could not create " + targetBaseDirPath + " with the required permissions");
|
||||
}
|
||||
}
|
||||
|
||||
private String getStoredAsClause() {
|
||||
if(config.tempTableStorageFormat.equalsIgnoreCase("json")) {
|
||||
// Special case for json
|
||||
// default json serde does not support having same key even if its under multiple depths
|
||||
return "ROW FORMAT SERDE 'org.openx.data.jsonserde.JsonSerDe' STORED AS TEXTFILE";
|
||||
}
|
||||
return "STORED AS " + config.tempTableStorageFormat;
|
||||
Path targetPath = new Path(targetBaseDirPath, lastCommitTime);
|
||||
if (fs.exists(targetPath)) {
|
||||
boolean result = fs.delete(targetPath, true);
|
||||
if (!result) {
|
||||
throw new HoodieException(
|
||||
"Could not delete existing " + targetPath);
|
||||
}
|
||||
}
|
||||
log.info("Creating " + targetPath + " with permission drwxrwxrwx");
|
||||
return FileSystem.mkdirs(fs, targetBaseDirPath,
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
}
|
||||
|
||||
private void initHiveBeelineProperties(Statement stmt) throws SQLException {
|
||||
log.info("Setting up Hive JDBC Session with properties");
|
||||
// set the queue
|
||||
executeStatement("set mapred.job.queue.name=" + config.yarnQueueName, stmt);
|
||||
// Set the inputformat to HoodieCombineHiveInputFormat
|
||||
executeStatement("set hive.input.format=com.uber.hoodie.hadoop.hive.HoodieCombineHiveInputFormat", stmt);
|
||||
// Allow queries without partition predicate
|
||||
executeStatement("set hive.strict.checks.large.query=false", stmt);
|
||||
// Dont gather stats for the table created
|
||||
executeStatement("set hive.stats.autogather=false", stmt);
|
||||
// Set the hoodie modie
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.mode=INCREMENTAL", stmt);
|
||||
// Set the from commit time
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.start.timestamp="
|
||||
+ config.fromCommitTime, stmt);
|
||||
// Set number of commits to pull
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String
|
||||
.valueOf(config.maxCommits), stmt);
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation)
|
||||
throws IOException {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
.filterCompletedInstants()
|
||||
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants()
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToSync.isEmpty()) {
|
||||
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata
|
||||
.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants()
|
||||
.getInstants()
|
||||
.collect(Collectors.toList()) + " and from commit time is "
|
||||
+ config.fromCommitTime);
|
||||
return null;
|
||||
}
|
||||
log.info("Syncing commits " + commitsToSync);
|
||||
return commitsToSync.get(commitsToSync.size() - 1);
|
||||
}
|
||||
|
||||
private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException {
|
||||
log.info("Deleting path " + path);
|
||||
return fs.delete(new Path(path), true);
|
||||
private Connection getConnection() throws SQLException {
|
||||
if (connection == null) {
|
||||
DataSource ds = getDatasource();
|
||||
log.info("Getting Hive Connection from Datasource " + ds);
|
||||
this.connection = ds.getConnection();
|
||||
}
|
||||
return connection;
|
||||
}
|
||||
|
||||
private void executeStatement(String sql, Statement stmt) throws SQLException {
|
||||
log.info("Executing: " + sql);
|
||||
stmt.execute(sql);
|
||||
}
|
||||
|
||||
private String inferCommitTime(FileSystem fs) throws SQLException, IOException {
|
||||
log.info("FromCommitTime not specified. Trying to infer it from Hoodie dataset "
|
||||
+ config.targetDb + "." + config.targetTable);
|
||||
String targetDataLocation = getTableLocation(config.targetDb, config.targetTable);
|
||||
return scanForCommitTime(fs, targetDataLocation);
|
||||
}
|
||||
|
||||
private String getTableLocation(String db, String table) throws SQLException {
|
||||
ResultSet resultSet = null;
|
||||
Statement stmt = null;
|
||||
try {
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
resultSet = stmt.executeQuery("describe formatted `" + db + "." + table + "`");
|
||||
while (resultSet.next()) {
|
||||
if (resultSet.getString(1).trim().equals("Location:")) {
|
||||
log.info("Inferred table location for " + db + "." + table + " as " + resultSet
|
||||
.getString(2));
|
||||
return resultSet.getString(2);
|
||||
}
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieIncrementalPullException(
|
||||
"Failed to get data location for table " + db + "." + table, e);
|
||||
} finally {
|
||||
try {
|
||||
if (stmt != null)
|
||||
stmt.close();
|
||||
if (resultSet != null)
|
||||
resultSet.close();
|
||||
} catch (SQLException e) {
|
||||
log.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IOException {
|
||||
if(targetDataPath == null) {
|
||||
throw new IllegalArgumentException("Please specify either --fromCommitTime or --targetDataPath");
|
||||
}
|
||||
if(!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
|
||||
return "0";
|
||||
}
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
|
||||
|
||||
Optional<HoodieInstant>
|
||||
lastCommit = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
if(lastCommit.isPresent()) {
|
||||
return lastCommit.get().getTimestamp();
|
||||
}
|
||||
return "0";
|
||||
}
|
||||
|
||||
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
|
||||
throws IOException {
|
||||
Path targetBaseDirPath = new Path(config.hoodieTmpDir, config.targetTable + "__" + config.sourceTable);
|
||||
if(!fs.exists(targetBaseDirPath)) {
|
||||
log.info("Creating " + targetBaseDirPath + " with permission drwxrwxrwx");
|
||||
boolean result = FileSystem.mkdirs(fs, targetBaseDirPath,
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
if (!result) {
|
||||
throw new HoodieException(
|
||||
"Could not create " + targetBaseDirPath + " with the required permissions");
|
||||
}
|
||||
}
|
||||
|
||||
Path targetPath = new Path(targetBaseDirPath, lastCommitTime);
|
||||
if(fs.exists(targetPath)) {
|
||||
boolean result = fs.delete(targetPath, true);
|
||||
if (!result) {
|
||||
throw new HoodieException(
|
||||
"Could not delete existing " + targetPath);
|
||||
}
|
||||
}
|
||||
log.info("Creating " + targetPath + " with permission drwxrwxrwx");
|
||||
return FileSystem.mkdirs(fs, targetBaseDirPath,
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
}
|
||||
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants()
|
||||
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToSync.isEmpty()) {
|
||||
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata
|
||||
.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants().getInstants()
|
||||
.collect(Collectors.toList()) + " and from commit time is "
|
||||
+ config.fromCommitTime);
|
||||
return null;
|
||||
}
|
||||
log.info("Syncing commits " + commitsToSync);
|
||||
return commitsToSync.get(commitsToSync.size() - 1);
|
||||
}
|
||||
|
||||
private Connection getConnection() throws SQLException {
|
||||
if (connection == null) {
|
||||
DataSource ds = getDatasource();
|
||||
log.info("Getting Hive Connection from Datasource " + ds);
|
||||
this.connection = ds.getConnection();
|
||||
}
|
||||
return connection;
|
||||
}
|
||||
|
||||
private DataSource getDatasource() {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(config.hiveJDBCUrl);
|
||||
ds.setUsername(config.hiveUsername);
|
||||
ds.setPassword(config.hivePassword);
|
||||
return ds;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
new HiveIncrementalPuller(cfg).saveDelta();
|
||||
private DataSource getDatasource() {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(config.hiveJDBCUrl);
|
||||
ds.setUsername(config.hiveUsername);
|
||||
ds.setPassword(config.hivePassword);
|
||||
return ds;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
new HiveIncrementalPuller(cfg).saveDelta();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,7 +20,6 @@ package com.uber.hoodie.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||
@@ -30,7 +29,12 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
@@ -39,140 +43,154 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Hoodie snapshot copy job which copies latest files from all partitions to another place, for snapshot backup.
|
||||
* Hoodie snapshot copy job which copies latest files from all partitions to another place, for
|
||||
* snapshot backup.
|
||||
*/
|
||||
public class HoodieSnapshotCopier implements Serializable {
|
||||
private static Logger logger = LogManager.getLogger(HoodieSnapshotCopier.class);
|
||||
|
||||
static class Config implements Serializable {
|
||||
@Parameter(names = {"--base-path", "-bp"}, description = "Hoodie table base path", required = true)
|
||||
String basePath = null;
|
||||
private static Logger logger = LogManager.getLogger(HoodieSnapshotCopier.class);
|
||||
|
||||
@Parameter(names = {"--output-path", "-op"}, description = "The snapshot output path", required = true)
|
||||
String outputPath = null;
|
||||
static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--date-partitioned", "-dp"}, description = "Can we assume date partitioning?")
|
||||
boolean shouldAssumeDatePartitioning = false;
|
||||
@Parameter(names = {"--base-path",
|
||||
"-bp"}, description = "Hoodie table base path", required = true)
|
||||
String basePath = null;
|
||||
|
||||
@Parameter(names = {"--output-path",
|
||||
"-op"}, description = "The snapshot output path", required = true)
|
||||
String outputPath = null;
|
||||
|
||||
@Parameter(names = {"--date-partitioned",
|
||||
"-dp"}, description = "Can we assume date partitioning?")
|
||||
boolean shouldAssumeDatePartitioning = false;
|
||||
}
|
||||
|
||||
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir,
|
||||
final boolean shouldAssumeDatePartitioning) throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView(
|
||||
tableMetadata,
|
||||
tableMetadata.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
.filterCompletedInstants());
|
||||
// Get the latest commit
|
||||
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline()
|
||||
.getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
if (!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
return;
|
||||
}
|
||||
final String latestCommitTimestamp = latestCommit.get().getTimestamp();
|
||||
logger.info(String
|
||||
.format("Starting to snapshot latest version files which are also no-late-than %s.",
|
||||
latestCommitTimestamp));
|
||||
|
||||
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, final boolean shouldAssumeDatePartitioning) throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView(tableMetadata,
|
||||
tableMetadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants());
|
||||
// Get the latest commit
|
||||
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline()
|
||||
.getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
if(!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
return;
|
||||
}
|
||||
final String latestCommitTimestamp = latestCommit.get().getTimestamp();
|
||||
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommitTimestamp));
|
||||
List<String> partitions = FSUtils
|
||||
.getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning);
|
||||
if (partitions.size() > 0) {
|
||||
logger.info(String.format("The job needs to copy %d partitions.", partitions.size()));
|
||||
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning);
|
||||
if (partitions.size() > 0) {
|
||||
logger.info(String.format("The job needs to copy %d partitions.", partitions.size()));
|
||||
// Make sure the output directory is empty
|
||||
Path outputPath = new Path(outputDir);
|
||||
if (fs.exists(outputPath)) {
|
||||
logger.warn(
|
||||
String.format("The output path %targetBasePath already exists, deleting", outputPath));
|
||||
fs.delete(new Path(outputDir), true);
|
||||
}
|
||||
|
||||
// Make sure the output directory is empty
|
||||
Path outputPath = new Path(outputDir);
|
||||
if (fs.exists(outputPath)) {
|
||||
logger.warn(String.format("The output path %targetBasePath already exists, deleting", outputPath));
|
||||
fs.delete(new Path(outputDir), true);
|
||||
jsc.parallelize(partitions, partitions.size())
|
||||
.flatMap(partition -> {
|
||||
// Only take latest version files <= latestCommit.
|
||||
FileSystem fs1 = FSUtils.getFs();
|
||||
List<Tuple2<String, String>> filePaths = new ArrayList<>();
|
||||
Stream<HoodieDataFile> dataFiles = fsView
|
||||
.getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp);
|
||||
dataFiles.forEach(
|
||||
hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
|
||||
|
||||
// also need to copy over partition metadata
|
||||
Path partitionMetaFile = new Path(new Path(baseDir, partition),
|
||||
HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
|
||||
if (fs1.exists(partitionMetaFile)) {
|
||||
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
|
||||
}
|
||||
|
||||
jsc.parallelize(partitions, partitions.size())
|
||||
.flatMap(partition -> {
|
||||
// Only take latest version files <= latestCommit.
|
||||
FileSystem fs1 = FSUtils.getFs();
|
||||
List<Tuple2<String, String>> filePaths = new ArrayList<>();
|
||||
Stream<HoodieDataFile> dataFiles = fsView.getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp);
|
||||
dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
|
||||
return filePaths.iterator();
|
||||
}).foreach(tuple -> {
|
||||
String partition = tuple._1();
|
||||
Path sourceFilePath = new Path(tuple._2());
|
||||
Path toPartitionPath = new Path(outputDir, partition);
|
||||
FileSystem fs1 = FSUtils.getFs();
|
||||
|
||||
// also need to copy over partition metadata
|
||||
Path partitionMetaFile = new Path(new Path(baseDir, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
|
||||
if (fs1.exists(partitionMetaFile)) {
|
||||
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
|
||||
}
|
||||
if (!fs1.exists(toPartitionPath)) {
|
||||
fs1.mkdirs(toPartitionPath);
|
||||
}
|
||||
FileUtil.copy(fs1, sourceFilePath, fs1,
|
||||
new Path(toPartitionPath, sourceFilePath.getName()), false, fs1.getConf());
|
||||
});
|
||||
|
||||
return filePaths.iterator();
|
||||
}).foreach(tuple -> {
|
||||
String partition = tuple._1();
|
||||
Path sourceFilePath = new Path(tuple._2());
|
||||
Path toPartitionPath = new Path(outputDir, partition);
|
||||
FileSystem fs1 = FSUtils.getFs();
|
||||
|
||||
if (!fs1.exists(toPartitionPath)) {
|
||||
fs1.mkdirs(toPartitionPath);
|
||||
}
|
||||
FileUtil.copy(fs1, sourceFilePath, fs1,
|
||||
new Path(toPartitionPath, sourceFilePath.getName()), false, fs1.getConf());
|
||||
});
|
||||
|
||||
// Also copy the .commit files
|
||||
logger.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
|
||||
FileStatus[] commitFilesToCopy = fs.listStatus(
|
||||
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), (commitFilePath) -> {
|
||||
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
|
||||
return true;
|
||||
} else {
|
||||
String commitTime =
|
||||
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return HoodieTimeline.compareTimestamps(commitTime, latestCommitTimestamp, HoodieTimeline.LESSER_OR_EQUAL);
|
||||
}
|
||||
});
|
||||
for (FileStatus commitStatus : commitFilesToCopy) {
|
||||
Path targetFilePath = new Path(
|
||||
outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus
|
||||
.getPath().getName());
|
||||
if (! fs.exists(targetFilePath.getParent())) {
|
||||
fs.mkdirs(targetFilePath.getParent());
|
||||
}
|
||||
if (fs.exists(targetFilePath)) {
|
||||
logger.error(String.format("The target output commit file (%targetBasePath) already exists.", targetFilePath));
|
||||
}
|
||||
FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf());
|
||||
// Also copy the .commit files
|
||||
logger.info(
|
||||
String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
|
||||
FileStatus[] commitFilesToCopy = fs.listStatus(
|
||||
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), (commitFilePath) -> {
|
||||
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
|
||||
return true;
|
||||
} else {
|
||||
String commitTime =
|
||||
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return HoodieTimeline.compareTimestamps(commitTime, latestCommitTimestamp,
|
||||
HoodieTimeline.LESSER_OR_EQUAL);
|
||||
}
|
||||
} else {
|
||||
logger.info("The job has 0 partition to copy.");
|
||||
});
|
||||
for (FileStatus commitStatus : commitFilesToCopy) {
|
||||
Path targetFilePath = new Path(
|
||||
outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus
|
||||
.getPath().getName());
|
||||
if (!fs.exists(targetFilePath.getParent())) {
|
||||
fs.mkdirs(targetFilePath.getParent());
|
||||
}
|
||||
|
||||
// Create the _SUCCESS tag
|
||||
Path successTagPath = new Path(outputDir + "/_SUCCESS");
|
||||
if (!fs.exists(successTagPath)) {
|
||||
logger.info("Creating _SUCCESS under targetBasePath: " + outputDir);
|
||||
fs.createNewFile(successTagPath);
|
||||
if (fs.exists(targetFilePath)) {
|
||||
logger.error(String
|
||||
.format("The target output commit file (%targetBasePath) already exists.",
|
||||
targetFilePath));
|
||||
}
|
||||
FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf());
|
||||
}
|
||||
} else {
|
||||
logger.info("The job has 0 partition to copy.");
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
// Take input configs
|
||||
final Config cfg = new Config();
|
||||
new JCommander(cfg, args);
|
||||
logger.info(String.format("Snapshot hoodie table from %targetBasePath to %targetBasePath", cfg.basePath, cfg.outputPath));
|
||||
|
||||
// Create a spark job to do the snapshot copy
|
||||
SparkConf sparkConf = new SparkConf().setAppName("Hoodie-snapshot-copier");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
logger.info("Initializing spark job.");
|
||||
|
||||
// Copy
|
||||
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
|
||||
copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning);
|
||||
|
||||
// Stop the job
|
||||
jsc.stop();
|
||||
// Create the _SUCCESS tag
|
||||
Path successTagPath = new Path(outputDir + "/_SUCCESS");
|
||||
if (!fs.exists(successTagPath)) {
|
||||
logger.info("Creating _SUCCESS under targetBasePath: " + outputDir);
|
||||
fs.createNewFile(successTagPath);
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
// Take input configs
|
||||
final Config cfg = new Config();
|
||||
new JCommander(cfg, args);
|
||||
logger.info(String
|
||||
.format("Snapshot hoodie table from %targetBasePath to %targetBasePath", cfg.basePath,
|
||||
cfg.outputPath));
|
||||
|
||||
// Create a spark job to do the snapshot copy
|
||||
SparkConf sparkConf = new SparkConf().setAppName("Hoodie-snapshot-copier");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
logger.info("Initializing spark job.");
|
||||
|
||||
// Copy
|
||||
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
|
||||
copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning);
|
||||
|
||||
// Stop the job
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,16 +18,12 @@
|
||||
|
||||
package com.uber.hoodie.utilities;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.KeyGenerator;
|
||||
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
import com.uber.hoodie.utilities.sources.Source;
|
||||
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
|
||||
import com.uber.hoodie.utilities.sources.SourceDataFormat;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import java.io.IOException;
|
||||
import org.apache.commons.configuration.ConfigurationException;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.commons.lang3.reflect.ConstructorUtils;
|
||||
@@ -36,50 +32,49 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Bunch of helper methods
|
||||
*/
|
||||
public class UtilHelpers {
|
||||
|
||||
public static Source createSource(String sourceClass, PropertiesConfiguration cfg, JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider) throws IOException {
|
||||
try {
|
||||
return (Source) ConstructorUtils.invokeConstructor(Class.forName(sourceClass), (Object) cfg, (Object) jssc, (Object) dataFormat, (Object) schemaProvider);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load source class " + sourceClass, e);
|
||||
}
|
||||
public static Source createSource(String sourceClass, PropertiesConfiguration cfg,
|
||||
JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider)
|
||||
throws IOException {
|
||||
try {
|
||||
return (Source) ConstructorUtils
|
||||
.invokeConstructor(Class.forName(sourceClass), (Object) cfg, (Object) jssc,
|
||||
(Object) dataFormat, (Object) schemaProvider);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load source class " + sourceClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static SchemaProvider createSchemaProvider(String schemaProviderClass, PropertiesConfiguration cfg) throws IOException {
|
||||
try {
|
||||
return (SchemaProvider) ConstructorUtils.invokeConstructor(Class.forName(schemaProviderClass), (Object) cfg);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load schema provider class " + schemaProviderClass, e);
|
||||
}
|
||||
public static SchemaProvider createSchemaProvider(String schemaProviderClass,
|
||||
PropertiesConfiguration cfg) throws IOException {
|
||||
try {
|
||||
return (SchemaProvider) ConstructorUtils
|
||||
.invokeConstructor(Class.forName(schemaProviderClass), (Object) cfg);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load schema provider class " + schemaProviderClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* TODO: Support hierarchical config files (see CONFIGURATION-609 for sample)
|
||||
*
|
||||
* @param fs
|
||||
* @param cfgPath
|
||||
* @return
|
||||
*/
|
||||
public static PropertiesConfiguration readConfig(FileSystem fs, Path cfgPath) {
|
||||
try {
|
||||
FSDataInputStream in = fs.open(cfgPath);
|
||||
PropertiesConfiguration config = new PropertiesConfiguration();
|
||||
config.load(in);
|
||||
in.close();
|
||||
return config;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Unable to read config file at :" + cfgPath, e);
|
||||
} catch (ConfigurationException e) {
|
||||
throw new HoodieDeltaStreamerException("Invalid configs found in config file at :" + cfgPath, e);
|
||||
}
|
||||
/**
|
||||
* TODO: Support hierarchical config files (see CONFIGURATION-609 for sample)
|
||||
*/
|
||||
public static PropertiesConfiguration readConfig(FileSystem fs, Path cfgPath) {
|
||||
try {
|
||||
FSDataInputStream in = fs.open(cfgPath);
|
||||
PropertiesConfiguration config = new PropertiesConfiguration();
|
||||
config.load(in);
|
||||
in.close();
|
||||
return config;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Unable to read config file at :" + cfgPath, e);
|
||||
} catch (ConfigurationException e) {
|
||||
throw new HoodieDeltaStreamerException("Invalid configs found in config file at :" + cfgPath,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -47,6 +47,13 @@ import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
import com.uber.hoodie.utilities.sources.DFSSource;
|
||||
import com.uber.hoodie.utilities.sources.Source;
|
||||
import com.uber.hoodie.utilities.sources.SourceDataFormat;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
@@ -60,299 +67,326 @@ import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.collection.JavaConversions;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* An Utility which can incrementally take the output from {@link HiveIncrementalPuller} and apply it to the target dataset.
|
||||
* Does not maintain any state, queries at runtime to see how far behind the target dataset is from
|
||||
* the source dataset. This can be overriden to force sync from a timestamp.
|
||||
* An Utility which can incrementally take the output from {@link HiveIncrementalPuller} and apply
|
||||
* it to the target dataset. Does not maintain any state, queries at runtime to see how far behind
|
||||
* the target dataset is from the source dataset. This can be overriden to force sync from a
|
||||
* timestamp.
|
||||
*/
|
||||
public class HoodieDeltaStreamer implements Serializable {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(HoodieDeltaStreamer.class);
|
||||
private static volatile Logger log = LogManager.getLogger(HoodieDeltaStreamer.class);
|
||||
|
||||
private static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
|
||||
private static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
|
||||
|
||||
private final Config cfg;
|
||||
private final Config cfg;
|
||||
|
||||
/**
|
||||
* Source to pull deltas from
|
||||
*/
|
||||
private transient Source source;
|
||||
/**
|
||||
* Source to pull deltas from
|
||||
*/
|
||||
private transient Source source;
|
||||
|
||||
/**
|
||||
* Schema provider that supplies the command for reading the input and writing out the
|
||||
* target table.
|
||||
*/
|
||||
private transient SchemaProvider schemaProvider;
|
||||
/**
|
||||
* Schema provider that supplies the command for reading the input and writing out the target
|
||||
* table.
|
||||
*/
|
||||
private transient SchemaProvider schemaProvider;
|
||||
|
||||
/**
|
||||
* Extract the key for the target dataset
|
||||
*/
|
||||
private KeyGenerator keyGenerator;
|
||||
/**
|
||||
* Extract the key for the target dataset
|
||||
*/
|
||||
private KeyGenerator keyGenerator;
|
||||
|
||||
/**
|
||||
* Filesystem used
|
||||
*/
|
||||
private transient FileSystem fs;
|
||||
/**
|
||||
* Filesystem used
|
||||
*/
|
||||
private transient FileSystem fs;
|
||||
|
||||
/**
|
||||
* Timeline with completed commits
|
||||
*/
|
||||
private transient Optional<HoodieTimeline> commitTimelineOpt;
|
||||
/**
|
||||
* Timeline with completed commits
|
||||
*/
|
||||
private transient Optional<HoodieTimeline> commitTimelineOpt;
|
||||
|
||||
/**
|
||||
* Spark context
|
||||
*/
|
||||
private transient JavaSparkContext jssc;
|
||||
/**
|
||||
* Spark context
|
||||
*/
|
||||
private transient JavaSparkContext jssc;
|
||||
|
||||
|
||||
public HoodieDeltaStreamer(Config cfg) throws IOException {
|
||||
this.cfg = cfg;
|
||||
this.fs = FSUtils.getFs();
|
||||
public HoodieDeltaStreamer(Config cfg) throws IOException {
|
||||
this.cfg = cfg;
|
||||
this.fs = FSUtils.getFs();
|
||||
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath);
|
||||
this.commitTimelineOpt = Optional
|
||||
.of(meta.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
.filterCompletedInstants());
|
||||
} else {
|
||||
this.commitTimelineOpt = Optional.empty();
|
||||
}
|
||||
|
||||
//TODO(vc) Should these be passed from outside?
|
||||
initSchemaProvider();
|
||||
initKeyGenerator();
|
||||
this.jssc = getSparkContext();
|
||||
|
||||
initSource();
|
||||
}
|
||||
|
||||
private void initSource() throws IOException {
|
||||
// Create the source & schema providers
|
||||
PropertiesConfiguration sourceCfg = UtilHelpers.readConfig(fs, new Path(cfg.sourceConfigProps));
|
||||
log.info("Creating source " + cfg.sourceClassName + " with configs : " + sourceCfg.toString());
|
||||
this.source = UtilHelpers
|
||||
.createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, schemaProvider);
|
||||
}
|
||||
|
||||
private void initSchemaProvider() throws IOException {
|
||||
PropertiesConfiguration schemaCfg = UtilHelpers
|
||||
.readConfig(fs, new Path(cfg.schemaProviderConfigProps));
|
||||
log.info(
|
||||
"Creating schema provider " + cfg.schemaProviderClassName + " with configs : " + schemaCfg
|
||||
.toString());
|
||||
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, schemaCfg);
|
||||
}
|
||||
|
||||
private void initKeyGenerator() throws IOException {
|
||||
PropertiesConfiguration keygenCfg = UtilHelpers.readConfig(fs, new Path(cfg.keyGeneratorProps));
|
||||
log.info("Creating key generator " + cfg.keyGeneratorClass + " with configs : " + keygenCfg
|
||||
.toString());
|
||||
this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, keygenCfg);
|
||||
}
|
||||
|
||||
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath);
|
||||
this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants());
|
||||
private JavaSparkContext getSparkContext() {
|
||||
SparkConf sparkConf = new SparkConf()
|
||||
.setAppName("hoodie-delta-streamer-" + cfg.targetTableName);
|
||||
//sparkConf.setMaster(cfg.sparkMaster);
|
||||
sparkConf.setMaster("local[2]");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
|
||||
// Configure hadoop conf
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec",
|
||||
"org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
// register the schemas, so that shuffle does not serialize the full schemas
|
||||
List<Schema> schemas = Arrays
|
||||
.asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema());
|
||||
sparkConf.registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList());
|
||||
return new JavaSparkContext(sparkConf);
|
||||
}
|
||||
|
||||
private void sync() throws Exception {
|
||||
// Retrieve the previous round checkpoints, if any
|
||||
Optional<String> resumeCheckpointStr = Optional.empty();
|
||||
if (commitTimelineOpt.isPresent()) {
|
||||
Optional<HoodieInstant> lastCommit = commitTimelineOpt.get().lastInstant();
|
||||
if (lastCommit.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
HoodieCommitMetadata
|
||||
.fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get());
|
||||
if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) {
|
||||
resumeCheckpointStr = Optional.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
|
||||
} else {
|
||||
this.commitTimelineOpt = Optional.empty();
|
||||
throw new HoodieDeltaStreamerException(
|
||||
"Unable to find previous checkpoint. Please double check if this table " +
|
||||
"was indeed built via delta streamer ");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName);
|
||||
HoodieTableMetaClient
|
||||
.initializePathAsHoodieDataset(FSUtils.getFs(), cfg.targetBasePath, properties);
|
||||
}
|
||||
log.info("Checkpoint to resume from : " + resumeCheckpointStr);
|
||||
|
||||
//TODO(vc) Should these be passed from outside?
|
||||
initSchemaProvider();
|
||||
initKeyGenerator();
|
||||
this.jssc = getSparkContext();
|
||||
// Pull the data from the source & prepare the write
|
||||
Pair<Optional<JavaRDD<GenericRecord>>, String> dataAndCheckpoint = source
|
||||
.fetchNewData(resumeCheckpointStr, cfg.maxInputBytes);
|
||||
|
||||
initSource();
|
||||
if (!dataAndCheckpoint.getKey().isPresent()) {
|
||||
log.info("No new data, nothing to commit.. ");
|
||||
return;
|
||||
}
|
||||
|
||||
private void initSource() throws IOException {
|
||||
// Create the source & schema providers
|
||||
PropertiesConfiguration sourceCfg = UtilHelpers.readConfig(fs, new Path(cfg.sourceConfigProps));
|
||||
log.info("Creating source " + cfg.sourceClassName + " with configs : " + sourceCfg.toString());
|
||||
this.source = UtilHelpers.createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, schemaProvider);
|
||||
JavaRDD<GenericRecord> avroRDD = dataAndCheckpoint.getKey().get();
|
||||
JavaRDD<HoodieRecord> records = avroRDD
|
||||
.map(gr -> {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(
|
||||
cfg.payloadClassName,
|
||||
gr,
|
||||
(Comparable) gr.get(cfg.sourceOrderingField));
|
||||
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
|
||||
});
|
||||
|
||||
// Perform the write
|
||||
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(cfg.hoodieClientProps);
|
||||
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg);
|
||||
String commitTime = client.startCommit();
|
||||
log.info("Starting commit : " + commitTime);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD;
|
||||
if (cfg.operation == Operation.INSERT) {
|
||||
writeStatusRDD = client.insert(records, commitTime);
|
||||
} else if (cfg.operation == Operation.UPSERT) {
|
||||
writeStatusRDD = client.upsert(records, commitTime);
|
||||
} else {
|
||||
throw new HoodieDeltaStreamerException("Unknown operation :" + cfg.operation);
|
||||
}
|
||||
|
||||
private void initSchemaProvider() throws IOException {
|
||||
PropertiesConfiguration schemaCfg = UtilHelpers.readConfig(fs, new Path(cfg.schemaProviderConfigProps));
|
||||
log.info("Creating schema provider " + cfg.schemaProviderClassName + " with configs : " + schemaCfg.toString());
|
||||
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, schemaCfg);
|
||||
// Simply commit for now. TODO(vc): Support better error handlers later on
|
||||
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
|
||||
checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue());
|
||||
|
||||
boolean success = client
|
||||
.commit(commitTime, writeStatusRDD, Optional.of(checkpointCommitMetadata));
|
||||
if (success) {
|
||||
log.info("Commit " + commitTime + " successful!");
|
||||
// TODO(vc): Kick off hive sync from here.
|
||||
|
||||
} else {
|
||||
log.info("Commit " + commitTime + " failed!");
|
||||
}
|
||||
client.close();
|
||||
}
|
||||
|
||||
private void initKeyGenerator() throws IOException {
|
||||
PropertiesConfiguration keygenCfg = UtilHelpers.readConfig(fs, new Path(cfg.keyGeneratorProps));
|
||||
log.info("Creating key generator " + cfg.keyGeneratorClass + " with configs : " + keygenCfg.toString());
|
||||
this.keyGenerator = DataSourceUtils.createKeyGenerator(cfg.keyGeneratorClass, keygenCfg);
|
||||
private HoodieWriteConfig getHoodieClientConfig(String hoodieClientCfgPath) throws Exception {
|
||||
return HoodieWriteConfig.newBuilder()
|
||||
.combineInput(true, true)
|
||||
.withPath(cfg.targetBasePath)
|
||||
.withAutoCommit(false)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withPayloadClass(OverwriteWithLatestAvroPayload.class.getName()).build())
|
||||
.withSchema(schemaProvider.getTargetSchema().toString())
|
||||
.forTable(cfg.targetTableName)
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.fromInputStream(fs.open(new Path(hoodieClientCfgPath)))
|
||||
.build();
|
||||
}
|
||||
|
||||
private enum Operation {
|
||||
UPSERT,
|
||||
INSERT
|
||||
}
|
||||
|
||||
private class OperationConvertor implements IStringConverter<Operation> {
|
||||
|
||||
@Override
|
||||
public Operation convert(String value) throws ParameterException {
|
||||
return Operation.valueOf(value);
|
||||
}
|
||||
}
|
||||
|
||||
private class SourceFormatConvertor implements IStringConverter<SourceDataFormat> {
|
||||
|
||||
private JavaSparkContext getSparkContext() {
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-delta-streamer-" + cfg.targetTableName);
|
||||
//sparkConf.setMaster(cfg.sparkMaster);
|
||||
sparkConf.setMaster("local[2]");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
|
||||
// Configure hadoop conf
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec",
|
||||
"org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
// register the schemas, so that shuffle does not serialize the full schemas
|
||||
List<Schema> schemas = Arrays.asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema());
|
||||
sparkConf.registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList());
|
||||
return new JavaSparkContext(sparkConf);
|
||||
@Override
|
||||
public SourceDataFormat convert(String value) throws ParameterException {
|
||||
return SourceDataFormat.valueOf(value);
|
||||
}
|
||||
}
|
||||
|
||||
private void sync() throws Exception {
|
||||
// Retrieve the previous round checkpoints, if any
|
||||
Optional<String> resumeCheckpointStr = Optional.empty();
|
||||
if (commitTimelineOpt.isPresent()) {
|
||||
Optional<HoodieInstant> lastCommit = commitTimelineOpt.get().lastInstant();
|
||||
if (lastCommit.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get());
|
||||
if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) {
|
||||
resumeCheckpointStr = Optional.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
|
||||
} else {
|
||||
throw new HoodieDeltaStreamerException("Unable to find previous checkpoint. Please double check if this table " +
|
||||
"was indeed built via delta streamer ");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName);
|
||||
HoodieTableMetaClient.initializePathAsHoodieDataset(FSUtils.getFs(), cfg.targetBasePath, properties);
|
||||
}
|
||||
log.info("Checkpoint to resume from : " + resumeCheckpointStr);
|
||||
public static class Config implements Serializable {
|
||||
|
||||
/**
|
||||
* TARGET CONFIGS
|
||||
**/
|
||||
@Parameter(names = {
|
||||
"--target-base-path"}, description = "base path for the target hoodie dataset", required = true)
|
||||
public String targetBasePath;
|
||||
|
||||
// TODO: How to obtain hive configs to register?
|
||||
@Parameter(names = {
|
||||
"--target-table"}, description = "name of the target table in Hive", required = true)
|
||||
public String targetTableName;
|
||||
|
||||
@Parameter(names = {"--hoodie-client-config"}, description =
|
||||
"path to properties file on localfs or dfs, with hoodie client config. Sane defaults" +
|
||||
"are used, but recommend use to provide basic things like metrics endpoints, hive configs etc")
|
||||
public String hoodieClientProps = null;
|
||||
|
||||
/**
|
||||
* SOURCE CONFIGS
|
||||
**/
|
||||
@Parameter(names = {"--source-class"}, description =
|
||||
"subclass of com.uber.hoodie.utilities.sources.Source to use to read data. " +
|
||||
"built-in options: com.uber.hoodie.utilities.common.{DFSSource (default), KafkaSource, HiveIncrPullSource}")
|
||||
public String sourceClassName = DFSSource.class.getName();
|
||||
|
||||
@Parameter(names = {"--source-config"}, description =
|
||||
"path to properties file on localfs or dfs, with source configs. " +
|
||||
"For list of acceptable properties, refer the source class", required = true)
|
||||
public String sourceConfigProps = null;
|
||||
|
||||
@Parameter(names = {"--source-format"}, description =
|
||||
"Format of data in source, JSON (default), Avro. All source data is " +
|
||||
"converted to Avro using the provided schema in any case", converter = SourceFormatConvertor.class)
|
||||
public SourceDataFormat sourceFormat = SourceDataFormat.JSON;
|
||||
|
||||
@Parameter(names = {"--source-ordering-field"}, description =
|
||||
"Field within source record to decide how to break ties between " +
|
||||
" records with same key in input data. Default: 'ts' holding unix timestamp of record")
|
||||
public String sourceOrderingField = "ts";
|
||||
|
||||
@Parameter(names = {"--key-generator-class"}, description =
|
||||
"Subclass of com.uber.hoodie.utilities.common.KeyExtractor to generate" +
|
||||
"a HoodieKey from the given avro record. Built in: SimpleKeyGenerator (Uses provided field names as recordkey & partitionpath. "
|
||||
+
|
||||
"Nested fields specified via dot notation, e.g: a.b.c)")
|
||||
public String keyGeneratorClass = SimpleKeyGenerator.class.getName();
|
||||
|
||||
@Parameter(names = {"--key-generator-config"}, description =
|
||||
"Path to properties file on localfs or dfs, with KeyGenerator configs. " +
|
||||
"For list of acceptable properites, refer the KeyGenerator class", required = true)
|
||||
public String keyGeneratorProps = null;
|
||||
|
||||
@Parameter(names = {"--payload-class"}, description =
|
||||
"subclass of HoodieRecordPayload, that works off a GenericRecord. " +
|
||||
"Default: SourceWrapperPayload. Implement your own, if you want to do something other than overwriting existing value")
|
||||
public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
|
||||
|
||||
@Parameter(names = {"--schemaprovider-class"}, description =
|
||||
"subclass of com.uber.hoodie.utilities.schema.SchemaProvider " +
|
||||
"to attach schemas to input & target table data, built in options: FilebasedSchemaProvider")
|
||||
public String schemaProviderClassName = FilebasedSchemaProvider.class.getName();
|
||||
|
||||
@Parameter(names = {"--schemaprovider-config"}, description =
|
||||
"path to properties file on localfs or dfs, with schema configs. " +
|
||||
"For list of acceptable properties, refer the schema provider class", required = true)
|
||||
public String schemaProviderConfigProps = null;
|
||||
|
||||
|
||||
// Pull the data from the source & prepare the write
|
||||
Pair<Optional<JavaRDD<GenericRecord>>, String> dataAndCheckpoint = source.fetchNewData(resumeCheckpointStr, cfg.maxInputBytes);
|
||||
/**
|
||||
* Other configs
|
||||
**/
|
||||
@Parameter(names = {
|
||||
"--max-input-bytes"}, description = "Maximum number of bytes to read from source. Default: 1TB")
|
||||
public long maxInputBytes = 1L * 1024 * 1024 * 1024 * 1024;
|
||||
|
||||
if (!dataAndCheckpoint.getKey().isPresent()) {
|
||||
log.info("No new data, nothing to commit.. ");
|
||||
return;
|
||||
}
|
||||
|
||||
JavaRDD<GenericRecord> avroRDD = dataAndCheckpoint.getKey().get();
|
||||
JavaRDD<HoodieRecord> records = avroRDD
|
||||
.map(gr -> {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(
|
||||
cfg.payloadClassName,
|
||||
gr,
|
||||
(Comparable) gr.get(cfg.sourceOrderingField));
|
||||
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
|
||||
});
|
||||
@Parameter(names = {"--op"}, description =
|
||||
"Takes one of these values : UPSERT (default), INSERT (use when input " +
|
||||
"is purely new data/inserts to gain speed)", converter = OperationConvertor.class)
|
||||
public Operation operation = Operation.UPSERT;
|
||||
|
||||
|
||||
// Perform the write
|
||||
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(cfg.hoodieClientProps);
|
||||
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg);
|
||||
String commitTime = client.startCommit();
|
||||
log.info("Starting commit : " + commitTime);
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD;
|
||||
if (cfg.operation == Operation.INSERT) {
|
||||
writeStatusRDD = client.insert(records, commitTime);
|
||||
} else if (cfg.operation == Operation.UPSERT) {
|
||||
writeStatusRDD = client.upsert(records, commitTime);
|
||||
} else {
|
||||
throw new HoodieDeltaStreamerException("Unknown operation :" + cfg.operation);
|
||||
}
|
||||
|
||||
// Simply commit for now. TODO(vc): Support better error handlers later on
|
||||
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
|
||||
checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue());
|
||||
|
||||
boolean success = client.commit(commitTime, writeStatusRDD, Optional.of(checkpointCommitMetadata));
|
||||
if (success) {
|
||||
log.info("Commit " + commitTime + " successful!");
|
||||
// TODO(vc): Kick off hive sync from here.
|
||||
|
||||
} else {
|
||||
log.info("Commit " + commitTime + " failed!");
|
||||
}
|
||||
client.close();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getHoodieClientConfig(String hoodieClientCfgPath) throws Exception {
|
||||
return HoodieWriteConfig.newBuilder()
|
||||
.combineInput(true, true)
|
||||
.withPath(cfg.targetBasePath)
|
||||
.withAutoCommit(false)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withPayloadClass(OverwriteWithLatestAvroPayload.class.getName()).build())
|
||||
.withSchema(schemaProvider.getTargetSchema().toString())
|
||||
.forTable(cfg.targetTableName)
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.fromInputStream(fs.open(new Path(hoodieClientCfgPath)))
|
||||
.build();
|
||||
}
|
||||
|
||||
private enum Operation {
|
||||
UPSERT,
|
||||
INSERT
|
||||
}
|
||||
|
||||
private class OperationConvertor implements IStringConverter<Operation> {
|
||||
@Override
|
||||
public Operation convert(String value) throws ParameterException {
|
||||
return Operation.valueOf(value);
|
||||
}
|
||||
}
|
||||
|
||||
private class SourceFormatConvertor implements IStringConverter<SourceDataFormat> {
|
||||
@Override
|
||||
public SourceDataFormat convert(String value) throws ParameterException {
|
||||
return SourceDataFormat.valueOf(value);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
/** TARGET CONFIGS **/
|
||||
@Parameter(names = {"--target-base-path"}, description = "base path for the target hoodie dataset", required = true)
|
||||
public String targetBasePath;
|
||||
|
||||
// TODO: How to obtain hive configs to register?
|
||||
@Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true)
|
||||
public String targetTableName;
|
||||
|
||||
@Parameter(names = {"--hoodie-client-config"}, description = "path to properties file on localfs or dfs, with hoodie client config. Sane defaults" +
|
||||
"are used, but recommend use to provide basic things like metrics endpoints, hive configs etc")
|
||||
public String hoodieClientProps = null;
|
||||
|
||||
/** SOURCE CONFIGS **/
|
||||
@Parameter(names = {"--source-class"}, description = "subclass of com.uber.hoodie.utilities.sources.Source to use to read data. " +
|
||||
"built-in options: com.uber.hoodie.utilities.common.{DFSSource (default), KafkaSource, HiveIncrPullSource}")
|
||||
public String sourceClassName = DFSSource.class.getName();
|
||||
|
||||
@Parameter(names = {"--source-config"}, description = "path to properties file on localfs or dfs, with source configs. " +
|
||||
"For list of acceptable properties, refer the source class", required = true)
|
||||
public String sourceConfigProps = null;
|
||||
|
||||
@Parameter(names = {"--source-format"}, description = "Format of data in source, JSON (default), Avro. All source data is " +
|
||||
"converted to Avro using the provided schema in any case", converter = SourceFormatConvertor.class)
|
||||
public SourceDataFormat sourceFormat = SourceDataFormat.JSON;
|
||||
|
||||
@Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how to break ties between " +
|
||||
" records with same key in input data. Default: 'ts' holding unix timestamp of record")
|
||||
public String sourceOrderingField = "ts";
|
||||
|
||||
@Parameter(names = {"--key-generator-class"}, description = "Subclass of com.uber.hoodie.utilities.common.KeyExtractor to generate" +
|
||||
"a HoodieKey from the given avro record. Built in: SimpleKeyGenerator (Uses provided field names as recordkey & partitionpath. " +
|
||||
"Nested fields specified via dot notation, e.g: a.b.c)")
|
||||
public String keyGeneratorClass = SimpleKeyGenerator.class.getName();
|
||||
|
||||
@Parameter(names = {"--key-generator-config"}, description = "Path to properties file on localfs or dfs, with KeyGenerator configs. " +
|
||||
"For list of acceptable properites, refer the KeyGenerator class", required = true)
|
||||
public String keyGeneratorProps = null;
|
||||
|
||||
@Parameter(names = {"--payload-class"}, description = "subclass of HoodieRecordPayload, that works off a GenericRecord. " +
|
||||
"Default: SourceWrapperPayload. Implement your own, if you want to do something other than overwriting existing value")
|
||||
public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
|
||||
|
||||
@Parameter(names = {"--schemaprovider-class"}, description = "subclass of com.uber.hoodie.utilities.schema.SchemaProvider " +
|
||||
"to attach schemas to input & target table data, built in options: FilebasedSchemaProvider")
|
||||
public String schemaProviderClassName = FilebasedSchemaProvider.class.getName();
|
||||
|
||||
@Parameter(names = {"--schemaprovider-config"}, description = "path to properties file on localfs or dfs, with schema configs. " +
|
||||
"For list of acceptable properties, refer the schema provider class", required = true)
|
||||
public String schemaProviderConfigProps = null;
|
||||
|
||||
|
||||
/** Other configs **/
|
||||
@Parameter(names = {"--max-input-bytes"}, description = "Maximum number of bytes to read from source. Default: 1TB")
|
||||
public long maxInputBytes = 1L * 1024 * 1024 * 1024 * 1024;
|
||||
|
||||
@Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input " +
|
||||
"is purely new data/inserts to gain speed)", converter = OperationConvertor.class)
|
||||
public Operation operation = Operation.UPSERT;
|
||||
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
new HoodieDeltaStreamer(cfg).sync();
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
new HoodieDeltaStreamer(cfg).sync();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,11 +21,12 @@ package com.uber.hoodie.utilities.exception;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
|
||||
public class HoodieDeltaStreamerException extends HoodieException {
|
||||
public HoodieDeltaStreamerException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieDeltaStreamerException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
public HoodieDeltaStreamerException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieDeltaStreamerException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,15 +19,15 @@
|
||||
package com.uber.hoodie.utilities.exception;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
|
||||
import java.sql.SQLException;
|
||||
|
||||
public class HoodieIncrementalPullException extends HoodieException {
|
||||
public HoodieIncrementalPullException(String msg, SQLException e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieIncrementalPullException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
public HoodieIncrementalPullException(String msg, SQLException e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieIncrementalPullException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,11 +21,12 @@ package com.uber.hoodie.utilities.exception;
|
||||
import java.sql.SQLException;
|
||||
|
||||
public class HoodieIncrementalPullSQLException extends HoodieIncrementalPullException {
|
||||
public HoodieIncrementalPullSQLException(String msg, SQLException e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieIncrementalPullSQLException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
public HoodieIncrementalPullSQLException(String msg, SQLException e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieIncrementalPullSQLException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,83 +23,86 @@ import com.uber.hoodie.SimpleKeyGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
|
||||
/**
|
||||
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
|
||||
*
|
||||
*/
|
||||
public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
enum TimestampType implements Serializable {
|
||||
UNIX_TIMESTAMP,
|
||||
DATE_STRING,
|
||||
MIXED
|
||||
enum TimestampType implements Serializable {
|
||||
UNIX_TIMESTAMP,
|
||||
DATE_STRING,
|
||||
MIXED
|
||||
}
|
||||
|
||||
private final TimestampType timestampType;
|
||||
|
||||
private SimpleDateFormat inputDateFormat;
|
||||
|
||||
private final String outputDateFormat;
|
||||
|
||||
|
||||
/**
|
||||
* Supported configs
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
// One value from TimestampType above
|
||||
private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type";
|
||||
private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformat";
|
||||
private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat";
|
||||
}
|
||||
|
||||
public TimestampBasedKeyGenerator(PropertiesConfiguration config) {
|
||||
super(config);
|
||||
DataSourceUtils.checkRequiredProperties(config,
|
||||
Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
|
||||
this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
|
||||
this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
|
||||
|
||||
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
|
||||
DataSourceUtils
|
||||
.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat = new SimpleDateFormat(
|
||||
config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
}
|
||||
}
|
||||
|
||||
private final TimestampType timestampType;
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
Object partitionVal = record.get(partitionPathField);
|
||||
SimpleDateFormat partitionPathFormat = new SimpleDateFormat(outputDateFormat);
|
||||
partitionPathFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
|
||||
private SimpleDateFormat inputDateFormat;
|
||||
try {
|
||||
long unixTime;
|
||||
if (partitionVal instanceof Double) {
|
||||
unixTime = ((Double) partitionVal).longValue();
|
||||
} else if (partitionVal instanceof Float) {
|
||||
unixTime = ((Float) partitionVal).longValue();
|
||||
} else if (partitionVal instanceof Long) {
|
||||
unixTime = (Long) partitionVal;
|
||||
} else if (partitionVal instanceof String) {
|
||||
unixTime = inputDateFormat.parse(partitionVal.toString()).getTime() / 1000;
|
||||
} else {
|
||||
throw new HoodieNotSupportedException(
|
||||
"Unexpected type for partition field: " + partitionVal.getClass().getName());
|
||||
}
|
||||
|
||||
private final String outputDateFormat;
|
||||
|
||||
|
||||
/**
|
||||
* Supported configs
|
||||
*/
|
||||
static class Config {
|
||||
// One value from TimestampType above
|
||||
private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type";
|
||||
private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformat";
|
||||
private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat";
|
||||
}
|
||||
|
||||
public TimestampBasedKeyGenerator(PropertiesConfiguration config) {
|
||||
super(config);
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
|
||||
this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
|
||||
this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
|
||||
|
||||
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat = new SimpleDateFormat(config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
Object partitionVal = record.get(partitionPathField);
|
||||
SimpleDateFormat partitionPathFormat = new SimpleDateFormat(outputDateFormat);
|
||||
partitionPathFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
|
||||
try {
|
||||
long unixTime;
|
||||
if (partitionVal instanceof Double) {
|
||||
unixTime = ((Double) partitionVal).longValue();
|
||||
} else if (partitionVal instanceof Float) {
|
||||
unixTime = ((Float) partitionVal).longValue();
|
||||
} else if (partitionVal instanceof Long) {
|
||||
unixTime = (Long) partitionVal;
|
||||
} else if (partitionVal instanceof String) {
|
||||
unixTime = inputDateFormat.parse(partitionVal.toString()).getTime() / 1000;
|
||||
} else {
|
||||
throw new HoodieNotSupportedException("Unexpected type for partition field: "+ partitionVal.getClass().getName());
|
||||
}
|
||||
|
||||
return new HoodieKey(record.get(recordKeyField).toString(),
|
||||
partitionPathFormat.format(new Date(unixTime * 1000)));
|
||||
} catch (ParseException pe) {
|
||||
throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, pe);
|
||||
}
|
||||
return new HoodieKey(record.get(recordKeyField).toString(),
|
||||
partitionPathFormat.format(new Date(unixTime * 1000)));
|
||||
} catch (ParseException pe) {
|
||||
throw new HoodieDeltaStreamerException(
|
||||
"Unable to parse input partition field :" + partitionVal, pe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,54 +21,56 @@ package com.uber.hoodie.utilities.schema;
|
||||
import com.uber.hoodie.DataSourceUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
/**
|
||||
* A simple schema provider, that reads off files on DFS
|
||||
*/
|
||||
public class FilebasedSchemaProvider extends SchemaProvider {
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.source.schema.file";
|
||||
private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.target.schema.file";
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.source.schema.file";
|
||||
private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.target.schema.file";
|
||||
}
|
||||
|
||||
private final FileSystem fs;
|
||||
|
||||
private final Schema sourceSchema;
|
||||
|
||||
private final Schema targetSchema;
|
||||
|
||||
public FilebasedSchemaProvider(PropertiesConfiguration config) {
|
||||
super(config);
|
||||
this.fs = FSUtils.getFs();
|
||||
|
||||
DataSourceUtils.checkRequiredProperties(config,
|
||||
Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP));
|
||||
try {
|
||||
this.sourceSchema = new Schema.Parser()
|
||||
.parse(fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP))));
|
||||
this.targetSchema = new Schema.Parser()
|
||||
.parse(fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP))));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading schema", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private final FileSystem fs;
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return sourceSchema;
|
||||
}
|
||||
|
||||
private final Schema sourceSchema;
|
||||
|
||||
private final Schema targetSchema;
|
||||
|
||||
public FilebasedSchemaProvider(PropertiesConfiguration config) {
|
||||
super(config);
|
||||
this.fs = FSUtils.getFs();
|
||||
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP));
|
||||
try {
|
||||
this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP))));
|
||||
this.targetSchema = new Schema.Parser().parse(fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP))));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading schema", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return sourceSchema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getTargetSchema() {
|
||||
return targetSchema;
|
||||
}
|
||||
@Override
|
||||
public Schema getTargetSchema() {
|
||||
return targetSchema;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,22 +18,22 @@
|
||||
|
||||
package com.uber.hoodie.utilities.schema;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Class to provide schema for reading data and also writing into a Hoodie table
|
||||
*/
|
||||
public abstract class SchemaProvider implements Serializable {
|
||||
|
||||
protected PropertiesConfiguration config;
|
||||
protected PropertiesConfiguration config;
|
||||
|
||||
protected SchemaProvider(PropertiesConfiguration config) {
|
||||
this.config = config;
|
||||
}
|
||||
protected SchemaProvider(PropertiesConfiguration config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
public abstract Schema getSourceSchema();
|
||||
public abstract Schema getSourceSchema();
|
||||
|
||||
public abstract Schema getTargetSchema();
|
||||
public abstract Schema getTargetSchema();
|
||||
}
|
||||
|
||||
@@ -21,73 +21,71 @@ package com.uber.hoodie.utilities.sources;
|
||||
import com.twitter.bijection.Injection;
|
||||
import com.twitter.bijection.avro.GenericAvroCodecs;
|
||||
import com.uber.hoodie.avro.MercifulJsonConverter;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Convert a variety of {@link SourceDataFormat} into Avro GenericRecords. Has a bunch of lazy fields
|
||||
* to circumvent issues around serializing these objects from driver to executors
|
||||
* Convert a variety of {@link SourceDataFormat} into Avro GenericRecords. Has a bunch of lazy
|
||||
* fields to circumvent issues around serializing these objects from driver to executors
|
||||
*/
|
||||
public class AvroConvertor implements Serializable {
|
||||
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient Schema schema;
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient Schema schema;
|
||||
|
||||
private final String schemaStr;
|
||||
private final String schemaStr;
|
||||
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient MercifulJsonConverter jsonConverter;
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient MercifulJsonConverter jsonConverter;
|
||||
|
||||
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient Injection<GenericRecord, byte[]> recordInjection;
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient Injection<GenericRecord, byte[]> recordInjection;
|
||||
|
||||
|
||||
public AvroConvertor(String schemaStr) {
|
||||
this.schemaStr = schemaStr;
|
||||
public AvroConvertor(String schemaStr) {
|
||||
this.schemaStr = schemaStr;
|
||||
}
|
||||
|
||||
|
||||
private void initSchema() {
|
||||
if (schema == null) {
|
||||
Schema.Parser parser = new Schema.Parser();
|
||||
schema = parser.parse(schemaStr);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void initSchema() {
|
||||
if (schema == null) {
|
||||
Schema.Parser parser = new Schema.Parser();
|
||||
schema = parser.parse(schemaStr);
|
||||
}
|
||||
private void initInjection() {
|
||||
if (recordInjection == null) {
|
||||
recordInjection = GenericAvroCodecs.toBinary(schema);
|
||||
}
|
||||
}
|
||||
|
||||
private void initInjection() {
|
||||
if (recordInjection == null) {
|
||||
recordInjection = GenericAvroCodecs.toBinary(schema);
|
||||
}
|
||||
}
|
||||
|
||||
private void initJsonConvertor() {
|
||||
if (jsonConverter == null) {
|
||||
jsonConverter = new MercifulJsonConverter(schema);
|
||||
}
|
||||
private void initJsonConvertor() {
|
||||
if (jsonConverter == null) {
|
||||
jsonConverter = new MercifulJsonConverter(schema);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public GenericRecord fromJson(String json) throws IOException {
|
||||
initSchema();
|
||||
initJsonConvertor();
|
||||
return jsonConverter.convert(json);
|
||||
}
|
||||
public GenericRecord fromJson(String json) throws IOException {
|
||||
initSchema();
|
||||
initJsonConvertor();
|
||||
return jsonConverter.convert(json);
|
||||
}
|
||||
|
||||
|
||||
public GenericRecord fromAvroBinary(byte[] avroBinary) throws IOException {
|
||||
initSchema();
|
||||
initInjection();
|
||||
return recordInjection.invert(avroBinary).get();
|
||||
}
|
||||
public GenericRecord fromAvroBinary(byte[] avroBinary) throws IOException {
|
||||
initSchema();
|
||||
initInjection();
|
||||
return recordInjection.invert(avroBinary).get();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,7 +23,12 @@ import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.mapred.AvroKey;
|
||||
import org.apache.avro.mapreduce.AvroKeyInputFormat;
|
||||
@@ -40,113 +45,121 @@ import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Source to read data from a given DFS directory structure, incrementally
|
||||
*/
|
||||
public class DFSSource extends Source {
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
|
||||
}
|
||||
|
||||
private final static List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
|
||||
|
||||
private final transient FileSystem fs;
|
||||
|
||||
public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
super(config, sparkContext, dataFormat, schemaProvider);
|
||||
this.fs = FSUtils.getFs();
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
|
||||
}
|
||||
|
||||
private final static List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
|
||||
|
||||
private final transient FileSystem fs;
|
||||
|
||||
public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext,
|
||||
SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
super(config, sparkContext, dataFormat, schemaProvider);
|
||||
this.fs = FSUtils.getFs();
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
||||
}
|
||||
|
||||
|
||||
public static JavaRDD<GenericRecord> fromAvroFiles(final AvroConvertor convertor, String pathStr,
|
||||
JavaSparkContext sparkContext) {
|
||||
JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr,
|
||||
AvroKeyInputFormat.class,
|
||||
AvroKey.class,
|
||||
NullWritable.class,
|
||||
sparkContext.hadoopConfiguration());
|
||||
return avroRDD.keys().map(r -> ((GenericRecord) r.datum()));
|
||||
}
|
||||
|
||||
public static JavaRDD<GenericRecord> fromJsonFiles(final AvroConvertor convertor, String pathStr,
|
||||
JavaSparkContext sparkContext) {
|
||||
return sparkContext.textFile(pathStr).map((String j) -> {
|
||||
return convertor.fromJson(j);
|
||||
});
|
||||
}
|
||||
|
||||
public static JavaRDD<GenericRecord> fromFiles(SourceDataFormat dataFormat,
|
||||
final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) {
|
||||
if (dataFormat == SourceDataFormat.AVRO) {
|
||||
return DFSSource.fromAvroFiles(convertor, pathStr, sparkContext);
|
||||
} else if (dataFormat == SourceDataFormat.JSON) {
|
||||
return DFSSource.fromJsonFiles(convertor, pathStr, sparkContext);
|
||||
} else {
|
||||
throw new HoodieNotSupportedException("Unsupported data format :" + dataFormat);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static JavaRDD<GenericRecord> fromAvroFiles(final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) {
|
||||
JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr,
|
||||
AvroKeyInputFormat.class,
|
||||
AvroKey.class,
|
||||
NullWritable.class,
|
||||
sparkContext.hadoopConfiguration());
|
||||
return avroRDD.keys().map(r -> ((GenericRecord) r.datum()));
|
||||
}
|
||||
@Override
|
||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(
|
||||
Optional<String> lastCheckpointStr, long maxInputBytes) {
|
||||
|
||||
public static JavaRDD<GenericRecord> fromJsonFiles(final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) {
|
||||
return sparkContext.textFile(pathStr).map((String j) -> {
|
||||
return convertor.fromJson(j);
|
||||
});
|
||||
}
|
||||
|
||||
public static JavaRDD<GenericRecord> fromFiles(SourceDataFormat dataFormat, final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) {
|
||||
if (dataFormat == SourceDataFormat.AVRO) {
|
||||
return DFSSource.fromAvroFiles(convertor, pathStr, sparkContext);
|
||||
} else if (dataFormat == SourceDataFormat.JSON) {
|
||||
return DFSSource.fromJsonFiles(convertor, pathStr, sparkContext);
|
||||
} else {
|
||||
throw new HoodieNotSupportedException("Unsupported data format :" + dataFormat);
|
||||
try {
|
||||
// obtain all eligible files under root folder.
|
||||
List<FileStatus> eligibleFiles = new ArrayList<>();
|
||||
RemoteIterator<LocatedFileStatus> fitr = fs
|
||||
.listFiles(new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true);
|
||||
while (fitr.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fitr.next();
|
||||
if (fileStatus.isDirectory() ||
|
||||
IGNORE_FILEPREFIX_LIST.stream()
|
||||
.filter(pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
eligibleFiles.add(fileStatus);
|
||||
}
|
||||
// sort them by modification time.
|
||||
eligibleFiles.sort((FileStatus f1, FileStatus f2) -> Long.valueOf(f1.getModificationTime())
|
||||
.compareTo(Long.valueOf(f2.getModificationTime())));
|
||||
|
||||
|
||||
@Override
|
||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr, long maxInputBytes) {
|
||||
|
||||
try {
|
||||
// obtain all eligible files under root folder.
|
||||
List<FileStatus> eligibleFiles = new ArrayList<>();
|
||||
RemoteIterator<LocatedFileStatus> fitr = fs.listFiles(new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true);
|
||||
while (fitr.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fitr.next();
|
||||
if (fileStatus.isDirectory() ||
|
||||
IGNORE_FILEPREFIX_LIST.stream().filter(pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) {
|
||||
continue;
|
||||
}
|
||||
eligibleFiles.add(fileStatus);
|
||||
}
|
||||
// sort them by modification time.
|
||||
eligibleFiles.sort((FileStatus f1, FileStatus f2) -> Long.valueOf(f1.getModificationTime()).compareTo(Long.valueOf(f2.getModificationTime())));
|
||||
|
||||
// Filter based on checkpoint & input size, if needed
|
||||
long currentBytes = 0;
|
||||
long maxModificationTime = Long.MIN_VALUE;
|
||||
List<FileStatus> filteredFiles = new ArrayList<>();
|
||||
for (FileStatus f : eligibleFiles) {
|
||||
if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long.valueOf(lastCheckpointStr.get())) {
|
||||
// skip processed files
|
||||
continue;
|
||||
}
|
||||
|
||||
maxModificationTime = f.getModificationTime();
|
||||
currentBytes += f.getLen();
|
||||
filteredFiles.add(f);
|
||||
if (currentBytes >= maxInputBytes) {
|
||||
// we have enough data, we are done
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// no data to read
|
||||
if (filteredFiles.size() == 0) {
|
||||
return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : String.valueOf(Long.MIN_VALUE));
|
||||
}
|
||||
|
||||
// read the files out.
|
||||
String pathStr = filteredFiles.stream().map(f -> f.getPath().toString()).collect(Collectors.joining(","));
|
||||
String schemaStr = schemaProvider.getSourceSchema().toString();
|
||||
final AvroConvertor avroConvertor = new AvroConvertor(schemaStr);
|
||||
|
||||
return new ImmutablePair<>(Optional.of(DFSSource.fromFiles(dataFormat, avroConvertor, pathStr, sparkContext)),
|
||||
String.valueOf(maxModificationTime));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Unable to read from source from checkpoint: " + lastCheckpointStr, ioe);
|
||||
// Filter based on checkpoint & input size, if needed
|
||||
long currentBytes = 0;
|
||||
long maxModificationTime = Long.MIN_VALUE;
|
||||
List<FileStatus> filteredFiles = new ArrayList<>();
|
||||
for (FileStatus f : eligibleFiles) {
|
||||
if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long
|
||||
.valueOf(lastCheckpointStr.get())) {
|
||||
// skip processed files
|
||||
continue;
|
||||
}
|
||||
|
||||
maxModificationTime = f.getModificationTime();
|
||||
currentBytes += f.getLen();
|
||||
filteredFiles.add(f);
|
||||
if (currentBytes >= maxInputBytes) {
|
||||
// we have enough data, we are done
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// no data to read
|
||||
if (filteredFiles.size() == 0) {
|
||||
return new ImmutablePair<>(Optional.empty(),
|
||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get()
|
||||
: String.valueOf(Long.MIN_VALUE));
|
||||
}
|
||||
|
||||
// read the files out.
|
||||
String pathStr = filteredFiles.stream().map(f -> f.getPath().toString())
|
||||
.collect(Collectors.joining(","));
|
||||
String schemaStr = schemaProvider.getSourceSchema().toString();
|
||||
final AvroConvertor avroConvertor = new AvroConvertor(schemaStr);
|
||||
|
||||
return new ImmutablePair<>(
|
||||
Optional.of(DFSSource.fromFiles(dataFormat, avroConvertor, pathStr, sparkContext)),
|
||||
String.valueOf(maxModificationTime));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(
|
||||
"Unable to read from source from checkpoint: " + lastCheckpointStr, ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,7 +22,13 @@ import com.uber.hoodie.DataSourceUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
@@ -35,98 +41,98 @@ import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit
|
||||
* by commit and apply to the target table
|
||||
*
|
||||
* The general idea here is to have commits sync across the data pipeline.
|
||||
*
|
||||
* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable
|
||||
* {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...}
|
||||
* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable
|
||||
* {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...}
|
||||
*
|
||||
* This produces beautiful causality, that makes data issues in ETLs very easy to debug
|
||||
*
|
||||
*/
|
||||
public class HiveIncrPullSource extends Source {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(HiveIncrPullSource.class);
|
||||
private static volatile Logger log = LogManager.getLogger(HiveIncrPullSource.class);
|
||||
|
||||
private final transient FileSystem fs;
|
||||
private final transient FileSystem fs;
|
||||
|
||||
private final String incrPullRootPath;
|
||||
private final String incrPullRootPath;
|
||||
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root";
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root";
|
||||
}
|
||||
|
||||
public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext,
|
||||
SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
super(config, sparkContext, dataFormat, schemaProvider);
|
||||
this.fs = FSUtils.getFs();
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
||||
this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the first commit from source, greater than the target's last commit, and reads it out.
|
||||
*/
|
||||
private Optional<String> findCommitToPull(Optional<String> latestTargetCommit)
|
||||
throws IOException {
|
||||
|
||||
log.info("Looking for commits ");
|
||||
|
||||
FileStatus[] commitTimePaths = fs.listStatus(new Path(incrPullRootPath));
|
||||
List<String> commitTimes = new ArrayList<>(commitTimePaths.length);
|
||||
for (FileStatus commitTimePath : commitTimePaths) {
|
||||
String[] splits = commitTimePath.getPath().toString().split("/");
|
||||
commitTimes.add(splits[splits.length - 1]);
|
||||
}
|
||||
Collections.sort(commitTimes);
|
||||
log.info("Retrieved commit times " + commitTimes);
|
||||
|
||||
if (!latestTargetCommit.isPresent()) {
|
||||
// start from the beginning
|
||||
return Optional.of(commitTimes.get(0));
|
||||
}
|
||||
|
||||
public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
super(config, sparkContext, dataFormat, schemaProvider);
|
||||
this.fs = FSUtils.getFs();
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
||||
this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP);
|
||||
for (String commitTime : commitTimes) {
|
||||
//TODO(vc): Add an option to delete consumed commits
|
||||
if (commitTime.compareTo(latestTargetCommit.get()) > 0) {
|
||||
return Optional.of(commitTime);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the first commit from source, greater than the target's last commit, and reads it out.
|
||||
*/
|
||||
private Optional<String> findCommitToPull(Optional<String> latestTargetCommit) throws IOException {
|
||||
@Override
|
||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(
|
||||
Optional<String> lastCheckpointStr, long maxInputBytes) {
|
||||
try {
|
||||
// find the source commit to pull
|
||||
Optional<String> commitToPull = findCommitToPull(lastCheckpointStr);
|
||||
|
||||
log.info("Looking for commits ");
|
||||
if (!commitToPull.isPresent()) {
|
||||
return new ImmutablePair<>(Optional.empty(),
|
||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
||||
}
|
||||
|
||||
FileStatus[] commitTimePaths = fs.listStatus(new Path(incrPullRootPath));
|
||||
List<String> commitTimes = new ArrayList<>(commitTimePaths.length);
|
||||
for (FileStatus commitTimePath : commitTimePaths) {
|
||||
String[] splits = commitTimePath.getPath().toString().split("/");
|
||||
commitTimes.add(splits[splits.length - 1]);
|
||||
}
|
||||
Collections.sort(commitTimes);
|
||||
log.info("Retrieved commit times " + commitTimes);
|
||||
|
||||
if (!latestTargetCommit.isPresent()) {
|
||||
// start from the beginning
|
||||
return Optional.of(commitTimes.get(0));
|
||||
}
|
||||
|
||||
for (String commitTime : commitTimes) {
|
||||
//TODO(vc): Add an option to delete consumed commits
|
||||
if (commitTime.compareTo(latestTargetCommit.get()) > 0) {
|
||||
return Optional.of(commitTime);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr, long maxInputBytes) {
|
||||
try {
|
||||
// find the source commit to pull
|
||||
Optional<String> commitToPull = findCommitToPull(lastCheckpointStr);
|
||||
|
||||
if (!commitToPull.isPresent()) {
|
||||
return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
||||
}
|
||||
|
||||
// read the files out.
|
||||
List<FileStatus> commitDeltaFiles = Arrays.asList(fs.listStatus(new Path(incrPullRootPath, commitToPull.get())));
|
||||
String pathStr = commitDeltaFiles.stream().map(f -> f.getPath().toString()).collect(Collectors.joining(","));
|
||||
String schemaStr = schemaProvider.getSourceSchema().toString();
|
||||
final AvroConvertor avroConvertor = new AvroConvertor(schemaStr);
|
||||
return new ImmutablePair<>(Optional.of(DFSSource.fromFiles(dataFormat, avroConvertor, pathStr, sparkContext)),
|
||||
String.valueOf(commitToPull.get()));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Unable to read from source from checkpoint: " + lastCheckpointStr, ioe);
|
||||
}
|
||||
// read the files out.
|
||||
List<FileStatus> commitDeltaFiles = Arrays
|
||||
.asList(fs.listStatus(new Path(incrPullRootPath, commitToPull.get())));
|
||||
String pathStr = commitDeltaFiles.stream().map(f -> f.getPath().toString())
|
||||
.collect(Collectors.joining(","));
|
||||
String schemaStr = schemaProvider.getSourceSchema().toString();
|
||||
final AvroConvertor avroConvertor = new AvroConvertor(schemaStr);
|
||||
return new ImmutablePair<>(
|
||||
Optional.of(DFSSource.fromFiles(dataFormat, avroConvertor, pathStr, sparkContext)),
|
||||
String.valueOf(commitToPull.get()));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(
|
||||
"Unable to read from source from checkpoint: " + lastCheckpointStr, ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,20 +22,6 @@ import com.uber.hoodie.DataSourceUtils;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException;
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.streaming.kafka.KafkaCluster;
|
||||
import org.apache.spark.streaming.kafka.KafkaUtils;
|
||||
import org.apache.spark.streaming.kafka.OffsetRange;
|
||||
import kafka.common.TopicAndPartition;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
@@ -48,8 +34,19 @@ import java.util.Spliterators;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import kafka.common.TopicAndPartition;
|
||||
import kafka.serializer.DefaultDecoder;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.streaming.kafka.KafkaCluster;
|
||||
import org.apache.spark.streaming.kafka.KafkaUtils;
|
||||
import org.apache.spark.streaming.kafka.OffsetRange;
|
||||
import scala.Predef;
|
||||
import scala.Tuple2;
|
||||
import scala.collection.JavaConverters;
|
||||
@@ -65,183 +62,192 @@ import scala.util.Either;
|
||||
*/
|
||||
public class KafkaSource extends Source {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(KafkaSource.class);
|
||||
private static volatile Logger log = LogManager.getLogger(KafkaSource.class);
|
||||
|
||||
|
||||
static class CheckpointUtils {
|
||||
static class CheckpointUtils {
|
||||
|
||||
/**
|
||||
* Reconstruct checkpoint from string.
|
||||
*
|
||||
* @param checkpointStr
|
||||
* @return
|
||||
*/
|
||||
public static HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> strToOffsets(String checkpointStr) {
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> offsetMap = new HashMap<>();
|
||||
String[] splits = checkpointStr.split(",");
|
||||
String topic = splits[0];
|
||||
for (int i = 1; i < splits.length; i++) {
|
||||
String[] subSplits = splits[i].split(":");
|
||||
offsetMap.put(new TopicAndPartition(topic, Integer.parseInt(subSplits[0])),
|
||||
new KafkaCluster.LeaderOffset("", -1, Long.parseLong(subSplits[1])));
|
||||
}
|
||||
return offsetMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* String representation of checkpoint
|
||||
*
|
||||
* Format:
|
||||
* topic1,0:offset0,1:offset1,2:offset2, .....
|
||||
*
|
||||
* @param offsetMap
|
||||
* @return
|
||||
*/
|
||||
public static String offsetsToStr(HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> offsetMap) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
// atleast 1 partition will be present.
|
||||
sb.append(offsetMap.entrySet().stream().findFirst().get().getKey().topic() + ",");
|
||||
sb.append(offsetMap.entrySet().stream()
|
||||
.map(e -> String.format("%s:%d",e.getKey().partition(), e.getValue().offset()))
|
||||
.collect(Collectors.joining(",")));
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public static OffsetRange[] computeOffsetRanges(HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> fromOffsetMap,
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> toOffsetMap) {
|
||||
Comparator<OffsetRange> byPartition = (OffsetRange o1, OffsetRange o2) -> {
|
||||
return Integer.valueOf(o1.partition()).compareTo(Integer.valueOf(o2.partition()));
|
||||
};
|
||||
List<OffsetRange> offsetRanges = toOffsetMap.entrySet().stream().map(e -> {
|
||||
TopicAndPartition tp = e.getKey();
|
||||
long fromOffset = -1;
|
||||
if (fromOffsetMap.containsKey(tp)){
|
||||
fromOffset = fromOffsetMap.get(tp).offset();
|
||||
}
|
||||
return OffsetRange.create(tp, fromOffset, e.getValue().offset());
|
||||
}).sorted(byPartition).collect(Collectors.toList());
|
||||
|
||||
OffsetRange[] ranges = new OffsetRange[offsetRanges.size()];
|
||||
return offsetRanges.toArray(ranges);
|
||||
}
|
||||
|
||||
public static long totalNewMessages(OffsetRange[] ranges) {
|
||||
long totalMsgs = 0;
|
||||
for (OffsetRange range: ranges) {
|
||||
totalMsgs += Math.max(range.untilOffset()-range.fromOffset(), 0);
|
||||
}
|
||||
return totalMsgs;
|
||||
}
|
||||
/**
|
||||
* Reconstruct checkpoint from string.
|
||||
*/
|
||||
public static HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> strToOffsets(
|
||||
String checkpointStr) {
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> offsetMap = new HashMap<>();
|
||||
String[] splits = checkpointStr.split(",");
|
||||
String topic = splits[0];
|
||||
for (int i = 1; i < splits.length; i++) {
|
||||
String[] subSplits = splits[i].split(":");
|
||||
offsetMap.put(new TopicAndPartition(topic, Integer.parseInt(subSplits[0])),
|
||||
new KafkaCluster.LeaderOffset("", -1, Long.parseLong(subSplits[1])));
|
||||
}
|
||||
return offsetMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Helpers to deal with tricky scala <=> java conversions. (oh my!)
|
||||
* String representation of checkpoint
|
||||
*
|
||||
* Format: topic1,0:offset0,1:offset1,2:offset2, .....
|
||||
*/
|
||||
static class ScalaHelpers {
|
||||
public static <K,V> Map<K, V> toScalaMap(HashMap<K, V> m) {
|
||||
return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(
|
||||
Predef.<Tuple2<K, V>>conforms()
|
||||
);
|
||||
}
|
||||
|
||||
public static Set<String> toScalaSet(HashSet<String> s) {
|
||||
return JavaConverters.asScalaSetConverter(s).asScala().<String>toSet();
|
||||
}
|
||||
|
||||
public static <K, V> java.util.Map<K, V> toJavaMap(Map<K, V> m) {
|
||||
return JavaConverters.<K, V>mapAsJavaMapConverter(m).asJava();
|
||||
}
|
||||
public static String offsetsToStr(
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> offsetMap) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
// atleast 1 partition will be present.
|
||||
sb.append(offsetMap.entrySet().stream().findFirst().get().getKey().topic() + ",");
|
||||
sb.append(offsetMap.entrySet().stream()
|
||||
.map(e -> String.format("%s:%d", e.getKey().partition(), e.getValue().offset()))
|
||||
.collect(Collectors.joining(",")));
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public static OffsetRange[] computeOffsetRanges(
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> fromOffsetMap,
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> toOffsetMap) {
|
||||
Comparator<OffsetRange> byPartition = (OffsetRange o1, OffsetRange o2) -> {
|
||||
return Integer.valueOf(o1.partition()).compareTo(Integer.valueOf(o2.partition()));
|
||||
};
|
||||
List<OffsetRange> offsetRanges = toOffsetMap.entrySet().stream().map(e -> {
|
||||
TopicAndPartition tp = e.getKey();
|
||||
long fromOffset = -1;
|
||||
if (fromOffsetMap.containsKey(tp)) {
|
||||
fromOffset = fromOffsetMap.get(tp).offset();
|
||||
}
|
||||
return OffsetRange.create(tp, fromOffset, e.getValue().offset());
|
||||
}).sorted(byPartition).collect(Collectors.toList());
|
||||
|
||||
/**
|
||||
* Configs to be passed for this source. All standard Kafka consumer configs are also
|
||||
* respected
|
||||
*/
|
||||
static class Config {
|
||||
private final static String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
|
||||
private final static String DEFAULT_AUTO_RESET_OFFSET = "largest";
|
||||
OffsetRange[] ranges = new OffsetRange[offsetRanges.size()];
|
||||
return offsetRanges.toArray(ranges);
|
||||
}
|
||||
|
||||
public static long totalNewMessages(OffsetRange[] ranges) {
|
||||
long totalMsgs = 0;
|
||||
for (OffsetRange range : ranges) {
|
||||
totalMsgs += Math.max(range.untilOffset() - range.fromOffset(), 0);
|
||||
}
|
||||
return totalMsgs;
|
||||
}
|
||||
}
|
||||
|
||||
private HashMap<String, String> kafkaParams;
|
||||
/**
|
||||
* Helpers to deal with tricky scala <=> java conversions. (oh my!)
|
||||
*/
|
||||
static class ScalaHelpers {
|
||||
|
||||
private final String topicName;
|
||||
|
||||
public KafkaSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
super(config, sparkContext, dataFormat, schemaProvider);
|
||||
|
||||
kafkaParams = new HashMap<>();
|
||||
Stream<String> keys = StreamSupport.stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false);
|
||||
keys.forEach(k -> kafkaParams.put(k, config.getString(k)));
|
||||
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME));
|
||||
topicName = config.getString(Config.KAFKA_TOPIC_NAME);
|
||||
public static <K, V> Map<K, V> toScalaMap(HashMap<K, V> m) {
|
||||
return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(
|
||||
Predef.<Tuple2<K, V>>conforms()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr, long maxInputBytes) {
|
||||
|
||||
// Obtain current metadata for the topic
|
||||
KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams));
|
||||
Either<ArrayBuffer<Throwable>, Set<TopicAndPartition>> either = cluster.getPartitions(ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName))));
|
||||
if (either.isLeft()) {
|
||||
// log errors. and bail out.
|
||||
throw new HoodieDeltaStreamerException("Error obtaining partition metadata", either.left().get().head());
|
||||
}
|
||||
Set<TopicAndPartition> topicPartitions = either.right().get();
|
||||
|
||||
// Determine the offset ranges to read from
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> fromOffsets;
|
||||
if (lastCheckpointStr.isPresent()) {
|
||||
fromOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get());
|
||||
} else {
|
||||
String autoResetValue = config.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET);
|
||||
if (autoResetValue.equals("smallest")) {
|
||||
fromOffsets = new HashMap(ScalaHelpers.toJavaMap(cluster.getEarliestLeaderOffsets(topicPartitions).right().get()));
|
||||
} else if (autoResetValue.equals("largest")) {
|
||||
fromOffsets = new HashMap(ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||
} else {
|
||||
throw new HoodieNotSupportedException("Auto reset value must be one of 'smallest' or 'largest' ");
|
||||
}
|
||||
}
|
||||
|
||||
// Always read until the latest offset
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> toOffsets = new HashMap(ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||
|
||||
|
||||
// Come up with final set of OffsetRanges to read (account for new partitions)
|
||||
// TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from partition size
|
||||
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets);
|
||||
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
|
||||
if (totalNewMsgs <= 0) {
|
||||
return new ImmutablePair<>(Optional.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : CheckpointUtils.offsetsToStr(toOffsets));
|
||||
} else {
|
||||
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName);
|
||||
}
|
||||
|
||||
|
||||
// Perform the actual read from Kafka
|
||||
JavaRDD<byte[]> kafkaRDD = KafkaUtils.createRDD(
|
||||
sparkContext,
|
||||
byte[].class,
|
||||
byte[].class,
|
||||
DefaultDecoder.class,
|
||||
DefaultDecoder.class,
|
||||
kafkaParams,
|
||||
offsetRanges).values();
|
||||
|
||||
// Produce a RDD[GenericRecord]
|
||||
final AvroConvertor avroConvertor = new AvroConvertor(schemaProvider.getSourceSchema().toString());
|
||||
JavaRDD<GenericRecord> newDataRDD;
|
||||
if (dataFormat == SourceDataFormat.AVRO) {
|
||||
newDataRDD = kafkaRDD.map(bytes -> avroConvertor.fromAvroBinary(bytes));
|
||||
} else if (dataFormat == SourceDataFormat.JSON) {
|
||||
newDataRDD = kafkaRDD.map(bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8"))));
|
||||
} else {
|
||||
throw new HoodieNotSupportedException("Unsupport data format :" + dataFormat);
|
||||
}
|
||||
|
||||
return new ImmutablePair<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(toOffsets));
|
||||
public static Set<String> toScalaSet(HashSet<String> s) {
|
||||
return JavaConverters.asScalaSetConverter(s).asScala().<String>toSet();
|
||||
}
|
||||
|
||||
public static <K, V> java.util.Map<K, V> toJavaMap(Map<K, V> m) {
|
||||
return JavaConverters.<K, V>mapAsJavaMapConverter(m).asJava();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Configs to be passed for this source. All standard Kafka consumer configs are also respected
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private final static String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
|
||||
private final static String DEFAULT_AUTO_RESET_OFFSET = "largest";
|
||||
}
|
||||
|
||||
|
||||
private HashMap<String, String> kafkaParams;
|
||||
|
||||
private final String topicName;
|
||||
|
||||
public KafkaSource(PropertiesConfiguration config, JavaSparkContext sparkContext,
|
||||
SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
super(config, sparkContext, dataFormat, schemaProvider);
|
||||
|
||||
kafkaParams = new HashMap<>();
|
||||
Stream<String> keys = StreamSupport
|
||||
.stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false);
|
||||
keys.forEach(k -> kafkaParams.put(k, config.getString(k)));
|
||||
|
||||
DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME));
|
||||
topicName = config.getString(Config.KAFKA_TOPIC_NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(
|
||||
Optional<String> lastCheckpointStr, long maxInputBytes) {
|
||||
|
||||
// Obtain current metadata for the topic
|
||||
KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams));
|
||||
Either<ArrayBuffer<Throwable>, Set<TopicAndPartition>> either = cluster
|
||||
.getPartitions(ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName))));
|
||||
if (either.isLeft()) {
|
||||
// log errors. and bail out.
|
||||
throw new HoodieDeltaStreamerException("Error obtaining partition metadata",
|
||||
either.left().get().head());
|
||||
}
|
||||
Set<TopicAndPartition> topicPartitions = either.right().get();
|
||||
|
||||
// Determine the offset ranges to read from
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> fromOffsets;
|
||||
if (lastCheckpointStr.isPresent()) {
|
||||
fromOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get());
|
||||
} else {
|
||||
String autoResetValue = config
|
||||
.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET);
|
||||
if (autoResetValue.equals("smallest")) {
|
||||
fromOffsets = new HashMap(ScalaHelpers
|
||||
.toJavaMap(cluster.getEarliestLeaderOffsets(topicPartitions).right().get()));
|
||||
} else if (autoResetValue.equals("largest")) {
|
||||
fromOffsets = new HashMap(
|
||||
ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||
} else {
|
||||
throw new HoodieNotSupportedException(
|
||||
"Auto reset value must be one of 'smallest' or 'largest' ");
|
||||
}
|
||||
}
|
||||
|
||||
// Always read until the latest offset
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> toOffsets = new HashMap(
|
||||
ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||
|
||||
// Come up with final set of OffsetRanges to read (account for new partitions)
|
||||
// TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from partition size
|
||||
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets);
|
||||
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
|
||||
if (totalNewMsgs <= 0) {
|
||||
return new ImmutablePair<>(Optional.empty(),
|
||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get()
|
||||
: CheckpointUtils.offsetsToStr(toOffsets));
|
||||
} else {
|
||||
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + topicName);
|
||||
}
|
||||
|
||||
// Perform the actual read from Kafka
|
||||
JavaRDD<byte[]> kafkaRDD = KafkaUtils.createRDD(
|
||||
sparkContext,
|
||||
byte[].class,
|
||||
byte[].class,
|
||||
DefaultDecoder.class,
|
||||
DefaultDecoder.class,
|
||||
kafkaParams,
|
||||
offsetRanges).values();
|
||||
|
||||
// Produce a RDD[GenericRecord]
|
||||
final AvroConvertor avroConvertor = new AvroConvertor(
|
||||
schemaProvider.getSourceSchema().toString());
|
||||
JavaRDD<GenericRecord> newDataRDD;
|
||||
if (dataFormat == SourceDataFormat.AVRO) {
|
||||
newDataRDD = kafkaRDD.map(bytes -> avroConvertor.fromAvroBinary(bytes));
|
||||
} else if (dataFormat == SourceDataFormat.JSON) {
|
||||
newDataRDD = kafkaRDD
|
||||
.map(bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8"))));
|
||||
} else {
|
||||
throw new HoodieNotSupportedException("Unsupport data format :" + dataFormat);
|
||||
}
|
||||
|
||||
return new ImmutablePair<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(toOffsets));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,50 +19,46 @@
|
||||
package com.uber.hoodie.utilities.sources;
|
||||
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Represents a source from which we can tail data. Assumes a constructor that takes properties.
|
||||
*/
|
||||
public abstract class Source implements Serializable {
|
||||
|
||||
protected transient PropertiesConfiguration config;
|
||||
protected transient PropertiesConfiguration config;
|
||||
|
||||
protected transient JavaSparkContext sparkContext;
|
||||
protected transient JavaSparkContext sparkContext;
|
||||
|
||||
protected transient SourceDataFormat dataFormat;
|
||||
protected transient SourceDataFormat dataFormat;
|
||||
|
||||
protected transient SchemaProvider schemaProvider;
|
||||
protected transient SchemaProvider schemaProvider;
|
||||
|
||||
|
||||
protected Source(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
this.config = config;
|
||||
this.sparkContext = sparkContext;
|
||||
this.dataFormat = dataFormat;
|
||||
this.schemaProvider = schemaProvider;
|
||||
}
|
||||
protected Source(PropertiesConfiguration config, JavaSparkContext sparkContext,
|
||||
SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
|
||||
this.config = config;
|
||||
this.sparkContext = sparkContext;
|
||||
this.dataFormat = dataFormat;
|
||||
this.schemaProvider = schemaProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches new data upto maxInputBytes, from the provided checkpoint and returns an RDD of the data,
|
||||
* as well as the checkpoint to be written as a result of that.
|
||||
*
|
||||
* @param lastCheckpointStr
|
||||
* @param maxInputBytes
|
||||
* @return
|
||||
*/
|
||||
public abstract Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr,
|
||||
long maxInputBytes);
|
||||
/**
|
||||
* Fetches new data upto maxInputBytes, from the provided checkpoint and returns an RDD of the
|
||||
* data, as well as the checkpoint to be written as a result of that.
|
||||
*/
|
||||
public abstract Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(
|
||||
Optional<String> lastCheckpointStr,
|
||||
long maxInputBytes);
|
||||
|
||||
|
||||
public PropertiesConfiguration getConfig() {
|
||||
return config;
|
||||
}
|
||||
public PropertiesConfiguration getConfig() {
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,8 +22,8 @@ package com.uber.hoodie.utilities.sources;
|
||||
* Format of the data within source.
|
||||
*/
|
||||
public enum SourceDataFormat {
|
||||
AVRO, // No conversion needed explicitly to avro
|
||||
JSON, // we will try to convert to avro
|
||||
ROW, // Will be added later, so we can plug/play with spark sources.
|
||||
CUSTOM // the source is responsible for conversion to avro.
|
||||
AVRO, // No conversion needed explicitly to avro
|
||||
JSON, // we will try to convert to avro
|
||||
ROW, // Will be added later, so we can plug/play with spark sources.
|
||||
CUSTOM // the source is responsible for conversion to avro.
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user