1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -18,8 +18,8 @@
package com.uber.hoodie;
import org.apache.avro.generic.GenericRecord;
import java.io.Serializable;
import org.apache.avro.generic.GenericRecord;
/**
* Base class for all AVRO record based payloads, that can be ordered based on a field
@@ -27,23 +27,23 @@ import java.io.Serializable;
public abstract class BaseAvroPayload implements Serializable {
/**
* Avro data extracted from the source
*/
protected final GenericRecord record;
/**
* Avro data extracted from the source
*/
protected final GenericRecord record;
/**
* For purposes of preCombining
*/
protected final Comparable orderingVal;
/**
* For purposes of preCombining
*/
protected final Comparable orderingVal;
/**
*
* @param record
* @param orderingVal
*/
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
this.record = record;
this.orderingVal = orderingVal;
}
/**
*
* @param record
* @param orderingVal
*/
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
this.record = record;
this.orderingVal = orderingVal;
}
}

View File

@@ -27,126 +27,130 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.configuration.PropertiesConfiguration;
import org.apache.commons.lang3.reflect.ConstructorUtils;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
* Utilities used throughout the data source
*/
public class DataSourceUtils {
/**
* Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c
*/
public static String getNestedFieldValAsString(GenericRecord record, String fieldName) {
String[] parts = fieldName.split("\\.");
GenericRecord valueNode = record;
for (int i = 0; i < parts.length; i++) {
String part = parts[i];
Object val = valueNode.get(part);
if (val == null) {
break;
}
/**
* Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c
*/
public static String getNestedFieldValAsString(GenericRecord record, String fieldName) {
String[] parts = fieldName.split("\\.");
GenericRecord valueNode = record;
for (int i = 0; i < parts.length; i++) {
String part = parts[i];
Object val = valueNode.get(part);
if (val == null) {
break;
}
// return, if last part of name
if (i == parts.length - 1) {
return val.toString();
} else {
// VC: Need a test here
if (!(val instanceof GenericRecord)) {
throw new HoodieException("Cannot find a record at part value :" + part);
}
valueNode = (GenericRecord) val;
}
// return, if last part of name
if (i == parts.length - 1) {
return val.toString();
} else {
// VC: Need a test here
if (!(val instanceof GenericRecord)) {
throw new HoodieException("Cannot find a record at part value :" + part);
}
throw new HoodieException(fieldName + " field not found in record");
valueNode = (GenericRecord) val;
}
}
throw new HoodieException(fieldName + " field not found in record");
}
/**
* Create a key generator class via reflection, passing in any configs needed
*/
public static KeyGenerator createKeyGenerator(String keyGeneratorClass, PropertiesConfiguration cfg) throws IOException {
try {
return (KeyGenerator) ConstructorUtils.invokeConstructor(Class.forName(keyGeneratorClass), (Object) cfg);
} catch (Throwable e) {
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
}
/**
* Create a key generator class via reflection, passing in any configs needed
*/
public static KeyGenerator createKeyGenerator(String keyGeneratorClass,
PropertiesConfiguration cfg) throws IOException {
try {
return (KeyGenerator) ConstructorUtils
.invokeConstructor(Class.forName(keyGeneratorClass), (Object) cfg);
} catch (Throwable e) {
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
}
}
/**
* Create a payload class via reflection, passing in an ordering/precombine value.
*/
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) throws IOException {
try {
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);
}
/**
* Create a payload class via reflection, passing in an ordering/precombine value.
*/
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record,
Comparable orderingVal) throws IOException {
try {
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);
}
}
public static void checkRequiredProperties(PropertiesConfiguration configuration, List<String> checkPropNames) {
checkPropNames.stream().forEach(prop -> {
if (!configuration.containsKey(prop)) {
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
}
});
public static void checkRequiredProperties(PropertiesConfiguration configuration,
List<String> checkPropNames) {
checkPropNames.stream().forEach(prop -> {
if (!configuration.containsKey(prop)) {
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
}
});
}
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()))
.build())
// override above with Hoodie configs specified as options.
.withProps(parameters)
.build();
return new HoodieWriteClient<>(jssc, writeConfig);
}
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client,
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())) {
return client.insert(hoodieRecords, commitTime);
} else {
//default is upsert
return client.upsert(hoodieRecords, commitTime);
}
}
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())).build())
// override above with Hoodie configs specified as options.
.withProps(parameters)
.build();
return new HoodieWriteClient<>(jssc, writeConfig);
}
public static JavaRDD<WriteStatus> doWriteOperation(HoodieWriteClient client,
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())) {
return client.insert(hoodieRecords, commitTime);
} else {
//default is upsert
return client.upsert(hoodieRecords, commitTime);
}
}
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);
}
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

@@ -19,68 +19,62 @@
package com.uber.hoodie;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
/**
* List of helpers to aid, construction of instanttime for read and write operations using datasource
* List of helpers to aid, construction of instanttime for read and write operations using
* datasource
*/
public class HoodieDataSourceHelpers {
/**
* Checks if the Hoodie dataset has new data since given timestamp. This can be subsequently
* fed to an incremental view read, to perform incremental processing.
*/
public static boolean hasNewCommits(FileSystem fs, String basePath, String commitTimestamp) {
return listCommitsSince(fs, basePath, commitTimestamp).size() > 0;
}
/**
* Checks if the Hoodie dataset has new data since given timestamp. This can be subsequently fed
* to an incremental view read, to perform incremental processing.
*/
public static boolean hasNewCommits(FileSystem fs, String basePath, String commitTimestamp) {
return listCommitsSince(fs, basePath, commitTimestamp).size() > 0;
}
/**
* Get a list of instant times that have occurred, from the given instant timestamp.
*
* @param instantTimestamp
*/
public static List<String> listCommitsSince(FileSystem fs, String basePath, String instantTimestamp) {
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
}
/**
* Get a list of instant times that have occurred, from the given instant timestamp.
*/
public static List<String> listCommitsSince(FileSystem fs, String basePath,
String instantTimestamp) {
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
}
/**
* Returns the last successful write operation's instant time
*/
public static String latestCommit(FileSystem fs, String basePath) {
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
return timeline.lastInstant().get().getTimestamp();
}
/**
* Returns the last successful write operation's instant time
*/
public static String latestCommit(FileSystem fs, String basePath) {
HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath);
return timeline.lastInstant().get().getTimestamp();
}
/**
* Obtain all the commits, compactions that have occurred on the timeline, whose
* instant times could be fed into the datasource options.
*
* @param fs
* @param basePath
*/
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
return table.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION)
);
} else {
return table.getCompletedCompactionCommitTimeline();
}
/**
* Obtain all the commits, compactions that have occurred on the timeline, whose instant times
* could be fed into the datasource options.
*/
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
return table.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION)
);
} else {
return table.getCompletedCompactionCommitTimeline();
}
}
}

View File

@@ -19,29 +19,24 @@
package com.uber.hoodie;
import com.uber.hoodie.common.model.HoodieKey;
import java.io.Serializable;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.configuration.PropertiesConfiguration;
import java.io.Serializable;
/**
* 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 {
protected transient PropertiesConfiguration config;
protected transient PropertiesConfiguration config;
protected KeyGenerator(PropertiesConfiguration config) {
this.config = config;
}
protected KeyGenerator(PropertiesConfiguration config) {
this.config = config;
}
/**
* Generate a Hoodie Key out of provided generic record.
*
* @param record
* @return
*/
public abstract HoodieKey getKey(GenericRecord record);
/**
* Generate a Hoodie Key out of provided generic record.
*/
public abstract HoodieKey getKey(GenericRecord record);
}

View File

@@ -20,49 +20,49 @@ package com.uber.hoodie;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import java.io.IOException;
import java.util.Optional;
/**
* Default payload used for delta streamer.
*
* 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
* 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
*/
public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
HoodieRecordPayload<OverwriteWithLatestAvroPayload> {
/**
*
* @param record
* @param orderingVal
*/
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
super(record, orderingVal);
}
/**
*
* @param record
* @param orderingVal
*/
public OverwriteWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
super(record, orderingVal);
}
@Override
public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
// pick the payload with greatest ordering value
if (another.orderingVal.compareTo(orderingVal) > 0) {
return another;
} else {
return this;
}
@Override
public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
// pick the payload with greatest ordering value
if (another.orderingVal.compareTo(orderingVal) > 0) {
return another;
} else {
return this;
}
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
// combining strategy here trivially ignores currentValue on disk and writes this record
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
// combining strategy here trivially ignores currentValue on disk and writes this record
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Optional.of(HoodieAvroUtils.rewriteRecord(record, schema));
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Optional.of(HoodieAvroUtils.rewriteRecord(record, schema));
}
}

View File

@@ -20,32 +20,33 @@ package com.uber.hoodie;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.exception.HoodieException;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.configuration.PropertiesConfiguration;
/**
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath
* as configs.
* Simple key generator, which takes names of fields to be used for recordKey and partitionPath as
* configs.
*/
public class SimpleKeyGenerator extends KeyGenerator {
protected final String recordKeyField;
protected final String recordKeyField;
protected final String partitionPathField;
protected final String partitionPathField;
public SimpleKeyGenerator(PropertiesConfiguration config) {
super(config);
this.recordKeyField = config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
this.partitionPathField = config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
}
@Override
public HoodieKey getKey(GenericRecord record) {
if (recordKeyField == null || partitionPathField == null) {
throw new HoodieException("Unable to find field names for record key or partition path in cfg");
}
return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
DataSourceUtils.getNestedFieldValAsString(record, partitionPathField));
public SimpleKeyGenerator(PropertiesConfiguration config) {
super(config);
this.recordKeyField = config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
this.partitionPathField = config
.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
}
@Override
public HoodieKey getKey(GenericRecord record) {
if (recordKeyField == null || partitionPathField == null) {
throw new HoodieException(
"Unable to find field names for record key or partition path in cfg");
}
return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
DataSourceUtils.getNestedFieldValAsString(record, partitionPathField));
}
}