1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -65,8 +65,7 @@ public class HDFSParquetImporter implements Serializable {
private transient FileSystem fs;
public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd");
public HDFSParquetImporter(
Config cfg) throws IOException {
public HDFSParquetImporter(Config cfg) throws IOException {
this.cfg = cfg;
}
@@ -77,9 +76,8 @@ public class HDFSParquetImporter implements Serializable {
@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));
throw new ParameterException(String.format(
"Invalid format type: value:%s: supported formats:%s", value, validFormats));
}
}
}
@@ -91,9 +89,8 @@ public class HDFSParquetImporter implements Serializable {
@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));
throw new ParameterException(String.format(
"Invalid source type: value:%s: supported source types:%s", value, validSourceTypes));
}
}
}
@@ -127,23 +124,21 @@ public class HDFSParquetImporter implements Serializable {
"-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)
"-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)
@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)
@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 {
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) {
@@ -182,8 +177,7 @@ public class HDFSParquetImporter implements Serializable {
// 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));
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);
@@ -205,8 +199,7 @@ public class HDFSParquetImporter implements Serializable {
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));
throw new HoodieIOException(String.format("Make sure %s is not present.", cfg.targetPath));
}
do {
ret = dataImport(jsc);
@@ -232,7 +225,8 @@ public class HDFSParquetImporter implements Serializable {
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(jsc.hadoopConfiguration(), cfg.targetPath, properties);
HoodieTableMetaClient
.initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties);
HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr,
cfg.parallelism);
@@ -240,50 +234,54 @@ public class HDFSParquetImporter implements Serializable {
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)));
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.
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(entry -> {
GenericRecord genericRecord = ((Tuple2<Void, 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<>(
new HoodieKey((String) rowField, partitionPath),
new HoodieJsonPayload(genericRecord.toString()));
}
);
GenericRecord genericRecord
= ((Tuple2<Void, 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<>(
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(writeStatus -> {
if (writeStatus.hasErrors()) {
errors.add(1);
logger.error(String.format("Error processing records :writeStatus:%s",
writeStatus.getStat().toString()));
}
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));
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()));

View File

@@ -52,9 +52,9 @@ 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.
*
* <p>
* Current Limitations:
*
* <p>
* - Only the source table can be incrementally pulled (usually the largest table) - The
* incrementally pulled table can't be referenced more than once.
*/
@@ -66,8 +66,7 @@ public class HiveIncrementalPuller {
public static class Config implements Serializable {
@Parameter(names = {"--hiveUrl"})
public String hiveJDBCUrl =
"jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2";
public String hiveJDBCUrl = "jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2";
@Parameter(names = {"--hiveUser"})
public String hiveUsername = "hive";
@Parameter(names = {"--hivePass"})
@@ -113,8 +112,8 @@ public class HiveIncrementalPuller {
public HiveIncrementalPuller(Config config) throws IOException {
this.config = config;
validateConfig(config);
String templateContent = IOUtils
.toString(this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
String templateContent = IOUtils.toString(
this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
incrementalPullSQLtemplate = new ST(templateContent);
}
@@ -182,19 +181,22 @@ public class HiveIncrementalPuller {
String storedAsClause = getStoredAsClause();
incrementalPullSQLtemplate.add("storedAsClause", storedAsClause);
String incrementalSQL =
new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z").next();
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.");
+ ", 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.");
+ " 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");
"Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which "
+ "means its not pulling incrementally");
}
incrementalPullSQLtemplate
@@ -227,11 +229,12 @@ public class HiveIncrementalPuller {
// 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);
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);
executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String.valueOf(
config.maxCommits), stmt);
}
private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException {
@@ -245,8 +248,9 @@ public class HiveIncrementalPuller {
}
private String inferCommitTime(FileSystem fs) throws SQLException, IOException {
log.info("FromCommitTime not specified. Trying to infer it from Hoodie dataset "
+ config.targetDb + "." + config.targetTable);
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);
}
@@ -260,8 +264,8 @@ public class HiveIncrementalPuller {
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));
log.info(
"Inferred table location for " + db + "." + table + " as " + resultSet.getString(2));
return resultSet.getString(2);
}
}
@@ -293,8 +297,7 @@ public class HiveIncrementalPuller {
}
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath);
Optional<HoodieInstant>
lastCommit = metadata.getActiveTimeline().getCommitsTimeline()
Optional<HoodieInstant> lastCommit = metadata.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants().lastInstant();
if (lastCommit.isPresent()) {
return lastCommit.get().getTimestamp();
@@ -302,8 +305,7 @@ public class HiveIncrementalPuller {
return "0";
}
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
throws IOException {
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) throws IOException {
Path targetBaseDirPath = new Path(config.hoodieTmpDir,
config.targetTable + "__" + config.sourceTable);
if (!fs.exists(targetBaseDirPath)) {
@@ -320,8 +322,7 @@ public class HiveIncrementalPuller {
if (fs.exists(targetPath)) {
boolean result = fs.delete(targetPath, true);
if (!result) {
throw new HoodieException(
"Could not delete existing " + targetPath);
throw new HoodieException("Could not delete existing " + targetPath);
}
}
log.info("Creating " + targetPath + " with permission drwxrwxrwx");
@@ -334,15 +335,14 @@ public class HiveIncrementalPuller {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation);
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants()
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants()
.map(HoodieInstant::getTimestamp)
.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().getCommitsTimeline().filterCompletedInstants()
.getInstants()
.collect(Collectors.toList()) + " and from commit time is "
+ config.fromCommitTime);
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are "
+ metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
.getInstants().collect(Collectors.toList())
+ " and from commit time is " + config.fromCommitTime);
return null;
}
log.info("Syncing commits " + commitsToSync);

View File

@@ -77,19 +77,18 @@ public class HoodieSnapshotCopier implements Serializable {
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir);
final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView(
tableMetadata,
tableMetadata.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants());
tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants());
// Get the latest commit
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline()
.getCommitsTimeline().filterCompletedInstants().lastInstant();
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline().getCommitsTimeline()
.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));
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);
@@ -104,25 +103,24 @@ public class HoodieSnapshotCopier implements Serializable {
fs.delete(new Path(outputDir), true);
}
jsc.parallelize(partitions, partitions.size())
.flatMap(partition -> {
// Only take latest version files <= latestCommit.
FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get());
List<Tuple2<String, String>> filePaths = new ArrayList<>();
Stream<HoodieDataFile> dataFiles = fsView
.getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp);
dataFiles.forEach(
hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
// Only take latest version files <= latestCommit.
FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get());
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()));
}
// 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()));
}
return filePaths.iterator();
}).foreach(tuple -> {
return filePaths.iterator();
}).foreach(tuple -> {
String partition = tuple._1();
Path sourceFilePath = new Path(tuple._2());
Path toPartitionPath = new Path(outputDir, partition);
@@ -131,8 +129,8 @@ public class HoodieSnapshotCopier implements Serializable {
if (!ifs.exists(toPartitionPath)) {
ifs.mkdirs(toPartitionPath);
}
FileUtil.copy(ifs, sourceFilePath, ifs,
new Path(toPartitionPath, sourceFilePath.getName()), false, ifs.getConf());
FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()),
false, ifs.getConf());
});
// Also copy the .commit files
@@ -143,23 +141,21 @@ public class HoodieSnapshotCopier implements Serializable {
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
return true;
} else {
String commitTime =
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
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());
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));
logger.error(String.format(
"The target output commit file (%targetBasePath) already exists.", targetFilePath));
}
FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf());
}
@@ -179,9 +175,8 @@ public class HoodieSnapshotCopier implements Serializable {
// 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));
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");

View File

@@ -41,9 +41,8 @@ public class UtilHelpers {
JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider)
throws IOException {
try {
return (Source) ConstructorUtils
.invokeConstructor(Class.forName(sourceClass), (Object) cfg, (Object) jssc,
(Object) dataFormat, (Object) schemaProvider);
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);
}
@@ -52,8 +51,8 @@ public class UtilHelpers {
public static SchemaProvider createSchemaProvider(String schemaProviderClass,
PropertiesConfiguration cfg) throws IOException {
try {
return (SchemaProvider) ConstructorUtils
.invokeConstructor(Class.forName(schemaProviderClass), (Object) cfg);
return (SchemaProvider) ConstructorUtils.invokeConstructor(Class.forName(schemaProviderClass),
(Object) cfg);
} catch (Throwable e) {
throw new IOException("Could not load schema provider class " + schemaProviderClass, e);
}

View File

@@ -120,9 +120,8 @@ public class HoodieDeltaStreamer implements Serializable {
if (fs.exists(new Path(cfg.targetBasePath))) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), cfg.targetBasePath);
this.commitTimelineOpt = Optional
.of(meta.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants());
this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants());
} else {
this.commitTimelineOpt = Optional.empty();
}
@@ -137,13 +136,13 @@ public class HoodieDeltaStreamer implements Serializable {
// 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);
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));
PropertiesConfiguration schemaCfg = UtilHelpers.readConfig(fs,
new Path(cfg.schemaProviderConfigProps));
log.info(
"Creating schema provider " + cfg.schemaProviderClassName + " with configs : " + schemaCfg
.toString());
@@ -175,8 +174,8 @@ public class HoodieDeltaStreamer implements Serializable {
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());
List<Schema> schemas = Arrays.asList(schemaProvider.getSourceSchema(),
schemaProvider.getTargetSchema());
sparkConf.registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList());
return new JavaSparkContext(sparkConf);
}
@@ -187,15 +186,14 @@ public class HoodieDeltaStreamer implements Serializable {
if (commitTimelineOpt.isPresent()) {
Optional<HoodieInstant> lastCommit = commitTimelineOpt.get().lastInstant();
if (lastCommit.isPresent()) {
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata
.fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get());
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 ");
"Unable to find previous checkpoint. Please double check if this table "
+ "was indeed built via delta streamer ");
}
}
} else {
@@ -208,8 +206,8 @@ public class HoodieDeltaStreamer implements Serializable {
log.info("Checkpoint to resume from : " + resumeCheckpointStr);
// Pull the data from the source & prepare the write
Pair<Optional<JavaRDD<GenericRecord>>, String> dataAndCheckpoint = source
.fetchNewData(resumeCheckpointStr, cfg.maxInputBytes);
Pair<Optional<JavaRDD<GenericRecord>>, String> dataAndCheckpoint = source.fetchNewData(
resumeCheckpointStr, cfg.maxInputBytes);
if (!dataAndCheckpoint.getKey().isPresent()) {
log.info("No new data, nothing to commit.. ");
@@ -217,14 +215,11 @@ public class HoodieDeltaStreamer implements Serializable {
}
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);
});
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);
@@ -245,8 +240,8 @@ public class HoodieDeltaStreamer implements Serializable {
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue());
boolean success = client
.commit(commitTime, writeStatusRDD, Optional.of(checkpointCommitMetadata));
boolean success = client.commit(commitTime, writeStatusRDD,
Optional.of(checkpointCommitMetadata));
if (success) {
log.info("Commit " + commitTime + " successful!");
// TODO(vc): Kick off hive sync from here.
@@ -258,23 +253,20 @@ public class HoodieDeltaStreamer implements Serializable {
}
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())
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(
.forTable(cfg.targetTableName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.fromInputStream(fs.open(new Path(hoodieClientCfgPath)))
.build();
.fromInputStream(fs.open(new Path(hoodieClientCfgPath))).build();
}
private enum Operation {
UPSERT,
INSERT
UPSERT, INSERT
}
private class OperationConvertor implements IStringConverter<Operation> {
@@ -308,58 +300,86 @@ public class HoodieDeltaStreamer implements Serializable {
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")
"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}")
"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)
"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)
"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")
"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)")
"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)
"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")
"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")
"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)
"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;
@@ -371,8 +391,9 @@ public class HoodieDeltaStreamer implements Serializable {
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)
"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;

View File

@@ -38,9 +38,7 @@ import org.apache.commons.configuration.PropertiesConfiguration;
public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
enum TimestampType implements Serializable {
UNIX_TIMESTAMP,
DATE_STRING,
MIXED
UNIX_TIMESTAMP, DATE_STRING, MIXED
}
private final TimestampType timestampType;
@@ -56,9 +54,14 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
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";
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) {

View File

@@ -39,8 +39,12 @@ public class FilebasedSchemaProvider extends SchemaProvider {
*/
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 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;
@@ -56,10 +60,10 @@ public class FilebasedSchemaProvider extends SchemaProvider {
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))));
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);
}

View File

@@ -55,10 +55,10 @@ public class DFSSource extends Source {
*/
static class Config {
private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
}
private final static List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
private static final List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
private final transient FileSystem fs;
@@ -73,9 +73,7 @@ public class DFSSource extends Source {
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,
AvroKeyInputFormat.class, AvroKey.class, NullWritable.class,
sparkContext.hadoopConfiguration());
return avroRDD.keys().map(r -> ((GenericRecord) r.datum()));
}
@@ -106,28 +104,28 @@ public class DFSSource extends Source {
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);
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) {
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())));
.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())) {
if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long.valueOf(
lastCheckpointStr.get())) {
// skip processed files
continue;
}

View File

@@ -44,12 +44,12 @@ import org.apache.spark.api.java.JavaSparkContext;
/**
* Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit
* by commit and apply to the target table
*
* <p>
* The general idea here is to have commits sync across the data pipeline.
*
* <p>
* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable
* {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...}
*
* <p>
* This produces beautiful causality, that makes data issues in ETLs very easy to debug
*/
public class HiveIncrPullSource extends Source {
@@ -66,7 +66,7 @@ public class HiveIncrPullSource extends Source {
*/
static class Config {
private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root";
private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root";
}
public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext,
@@ -121,8 +121,8 @@ public class HiveIncrPullSource extends Source {
}
// read the files out.
List<FileStatus> commitDeltaFiles = Arrays
.asList(fs.listStatus(new Path(incrPullRootPath, commitToPull.get())));
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();

View File

@@ -84,7 +84,7 @@ public class KafkaSource extends Source {
/**
* String representation of checkpoint
*
* <p>
* Format: topic1,0:offset0,1:offset1,2:offset2, .....
*/
public static String offsetsToStr(
@@ -132,9 +132,7 @@ public class KafkaSource extends Source {
static class ScalaHelpers {
public static <K, V> Map<K, V> toScalaMap(HashMap<K, V> m) {
return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(
Predef.conforms()
);
return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(Predef.conforms());
}
public static Set<String> toScalaSet(HashSet<String> s) {
@@ -152,8 +150,8 @@ public class KafkaSource extends Source {
*/
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 static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
private static final String DEFAULT_AUTO_RESET_OFFSET = "largest";
}
@@ -166,8 +164,8 @@ public class KafkaSource extends Source {
super(config, sparkContext, dataFormat, schemaProvider);
kafkaParams = new HashMap<>();
Stream<String> keys = StreamSupport
.stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false);
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));
@@ -180,8 +178,8 @@ public class KafkaSource extends Source {
// 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))));
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",
@@ -197,8 +195,8 @@ public class KafkaSource extends Source {
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()));
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()));
@@ -213,7 +211,8 @@ public class KafkaSource extends Source {
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
// 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) {
@@ -225,14 +224,8 @@ public class KafkaSource extends Source {
}
// Perform the actual read from Kafka
JavaRDD<byte[]> kafkaRDD = KafkaUtils.createRDD(
sparkContext,
byte[].class,
byte[].class,
DefaultDecoder.class,
DefaultDecoder.class,
kafkaParams,
offsetRanges).values();
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(
@@ -241,8 +234,8 @@ public class KafkaSource extends Source {
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"))));
newDataRDD = kafkaRDD.map(
bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8"))));
} else {
throw new HoodieNotSupportedException("Unsupport data format :" + dataFormat);
}

View File

@@ -54,8 +54,7 @@ public abstract class Source implements Serializable {
* 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);
Optional<String> lastCheckpointStr, long maxInputBytes);
public PropertiesConfiguration getConfig() {