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

@@ -38,7 +38,6 @@ public abstract class BaseAvroPayload implements Serializable {
protected final Comparable orderingVal;
/**
*
* @param record
* @param orderingVal
*/

View File

@@ -87,8 +87,8 @@ public class DataSourceUtils {
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record,
Comparable orderingVal) throws IOException {
try {
return (HoodieRecordPayload) ConstructorUtils
.invokeConstructor(Class.forName(payloadClass), (Object) record, (Object) orderingVal);
return (HoodieRecordPayload) ConstructorUtils.invokeConstructor(Class.forName(payloadClass),
(Object) record, (Object) orderingVal);
} catch (Throwable e) {
throw new IOException("Could not create payload for class: " + payloadClass, e);
}
@@ -103,36 +103,26 @@ public class DataSourceUtils {
});
}
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc,
String schemaStr,
String basePath,
String tblName,
Map<String, String> parameters) throws Exception {
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
.combineInput(true, true)
.withPath(basePath)
.withAutoCommit(false)
.withSchema(schemaStr)
.forTable(tblName)
.withIndexConfig(
HoodieIndexConfig.newBuilder()
.withIndexType(HoodieIndex.IndexType.BLOOM)
.build())
public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr,
String basePath, String tblName, Map<String, String> parameters) throws Exception {
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().combineInput(true, true)
.withPath(basePath).withAutoCommit(false)
.withSchema(schemaStr).forTable(tblName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY()))
.withPayloadClass(parameters.get(
DataSourceWriteOptions
.PAYLOAD_CLASS_OPT_KEY()))
.build())
// override above with Hoodie configs specified as options.
.withProps(parameters)
.build();
.withProps(parameters).build();
return new HoodieWriteClient<>(jssc, writeConfig);
}
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client,
JavaRDD<HoodieRecord> hoodieRecords,
String commitTime,
String operation) {
JavaRDD<HoodieRecord> hoodieRecords, String commitTime, String operation) {
if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) {
return client.bulkInsert(hoodieRecords, commitTime);
} else if (operation.equals(DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())) {
@@ -143,14 +133,9 @@ public class DataSourceUtils {
}
}
public static HoodieRecord createHoodieRecord(GenericRecord gr,
Comparable orderingVal,
HoodieKey hKey,
String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(
payloadClass,
gr,
orderingVal);
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal,
HoodieKey hKey, String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
return new HoodieRecord<>(hKey, payload);
}
}

View File

@@ -67,11 +67,12 @@ public class HoodieDataSourceHelpers {
*/
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true),
null);
if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
return table.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION));
HoodieActiveTimeline.DELTA_COMMIT_ACTION));
} else {
return table.getCommitTimeline().filterCompletedInstants();
}

View File

@@ -24,7 +24,8 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.commons.configuration.PropertiesConfiguration;
/**
* Abstract class to extend for plugging in extraction of {@link com.uber.hoodie.common.model.HoodieKey}
* Abstract class to extend for plugging in extraction of
* {@link com.uber.hoodie.common.model.HoodieKey}
* from an Avro record
*/
public abstract class KeyGenerator implements Serializable {

View File

@@ -28,7 +28,7 @@ import org.apache.avro.generic.IndexedRecord;
/**
* Default payload used for delta streamer.
*
* <p>
* 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2.
* combineAndGetUpdateValue/getInsertValue - Simply overwrites storage with latest delta record
*/
@@ -36,7 +36,6 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
/**
*
* @param record
* @param orderingVal
*/

View File

@@ -47,10 +47,11 @@ object AvroConversionUtils {
structName: String,
recordNamespace: String): (Any) => Any = {
dataType match {
case BinaryType => (item: Any) => item match {
case null => null
case bytes: Array[Byte] => ByteBuffer.wrap(bytes)
}
case BinaryType => (item: Any) =>
item match {
case null => null
case bytes: Array[Byte] => ByteBuffer.wrap(bytes)
}
case ByteType | ShortType | IntegerType | LongType |
FloatType | DoubleType | StringType | BooleanType => identity
case _: DecimalType => (item: Any) => if (item == null) null else item.toString

View File

@@ -32,17 +32,15 @@ public class DataSourceTestUtils {
try {
String str = ((TestRawTripPayload) record.getData()).getJsonData();
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
return Optional
.of(str.replaceAll("}", ", \"partition\": \"" + record.getPartitionPath() + "\"}"));
return Optional.of(str.replaceAll("}",
", \"partition\": \"" + record.getPartitionPath() + "\"}"));
} catch (IOException e) {
return Optional.empty();
}
}
public static List<String> convertToStringList(List<HoodieRecord> records) {
return records.stream().map(hr -> convertToString(hr))
.filter(os -> os.isPresent())
.map(os -> os.get())
.collect(Collectors.toList());
return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent())
.map(os -> os.get()).collect(Collectors.toList());
}
}

View File

@@ -1,12 +1,12 @@
/*
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
*
* http://www.apache.org/licenses/LICENSE-2.0
*
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -16,7 +16,6 @@
*
*/
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.uber.hoodie.DataSourceReadOptions;
@@ -68,10 +67,9 @@ public class HoodieJavaApp {
public void run() throws Exception {
// Spark session setup..
SparkSession spark = SparkSession.builder()
.appName("Hoodie Spark APP")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.master("local[1]")
SparkSession spark = SparkSession.builder().appName("Hoodie Spark APP")
.config("spark.serializer",
"org.apache.spark.serializer.KryoSerializer").master("local[1]")
.getOrCreate();
JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext());
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
@@ -83,13 +81,12 @@ public class HoodieJavaApp {
* Commit with only inserts
*/
// Generate some input..
List<String> records1 = DataSourceTestUtils
.convertToStringList(dataGen.generateInserts("001"/* ignore */, 100));
List<String> records1 = DataSourceTestUtils.convertToStringList(
dataGen.generateInserts("001"/* ignore */, 100));
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
// Save as hoodie dataset (copy on write)
inputDF1.write()
.format("com.uber.hoodie") // specify the hoodie source
inputDF1.write().format("com.uber.hoodie") // specify the hoodie source
.option("hoodie.insert.shuffle.parallelism",
"2") // any hoodie client config can be passed like this
.option("hoodie.upsert.shuffle.parallelism",
@@ -104,7 +101,8 @@ public class HoodieJavaApp {
"timestamp") // use to combine duplicate records in input/with disk val
.option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries
.mode(
SaveMode.Overwrite) // This will remove any existing data at path below, and create a new dataset if needed
SaveMode.Overwrite) // This will remove any existing data at path below, and create a
// new dataset if needed
.save(tablePath); // ultimately where the dataset will be placed
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
logger.info("First commit at instant time :" + commitInstantTime1);
@@ -112,28 +110,24 @@ public class HoodieJavaApp {
/**
* Commit that updates records
*/
List<String> records2 = DataSourceTestUtils
.convertToStringList(dataGen.generateUpdates("002"/* ignore */, 100));
List<String> records2 = DataSourceTestUtils.convertToStringList(
dataGen.generateUpdates("002"/* ignore */, 100));
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
inputDF2.write()
.format("com.uber.hoodie")
.option("hoodie.insert.shuffle.parallelism", "2")
inputDF2.write().format("com.uber.hoodie").option("hoodie.insert.shuffle.parallelism", "2")
.option("hoodie.upsert.shuffle.parallelism", "2")
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(HoodieWriteConfig.TABLE_NAME, tableName)
.mode(SaveMode.Append)
.save(tablePath);
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append).save(tablePath);
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
logger.info("Second commit at instant time :" + commitInstantTime1);
/**
* Read & do some queries
*/
Dataset<Row> hoodieROViewDF = spark.read()
.format("com.uber.hoodie")
// pass any path glob, can include hoodie & non-hoodie datasets
Dataset<Row> hoodieROViewDF = spark.read().format("com.uber.hoodie")
// pass any path glob, can include hoodie & non-hoodie
// datasets
.load(tablePath + "/*/*/*/*");
hoodieROViewDF.registerTempTable("hoodie_ro");
spark.sql("describe hoodie_ro").show();
@@ -149,7 +143,8 @@ public class HoodieJavaApp {
DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(),
commitInstantTime1) // Only changes in write 2 above
.load(tablePath); // For incremental view, pass in the root/base path of dataset
.load(
tablePath); // For incremental view, pass in the root/base path of dataset
logger.info("You will only see records from : " + commitInstantTime2);
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();