Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -115,12 +115,12 @@ object AvroConversionUtils {
|
||||
|
||||
def convertStructTypeToAvroSchema(structType: StructType,
|
||||
structName: String,
|
||||
recordNamespace: String) : Schema = {
|
||||
recordNamespace: String): Schema = {
|
||||
val builder = SchemaBuilder.record(structName).namespace(recordNamespace)
|
||||
SchemaConverters.convertStructToAvro(structType, builder, recordNamespace)
|
||||
}
|
||||
|
||||
def convertAvroSchemaToStructType(avroSchema: Schema) : StructType = {
|
||||
def convertAvroSchemaToStructType(avroSchema: Schema): StructType = {
|
||||
SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType];
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package com.uber.hoodie
|
||||
import org.apache.spark.sql.{DataFrame, DataFrameReader, DataFrameWriter}
|
||||
|
||||
package object hoodie {
|
||||
|
||||
/**
|
||||
* Adds a method, `hoodie`, to DataFrameWriter
|
||||
*/
|
||||
@@ -33,4 +34,5 @@ package object hoodie {
|
||||
implicit class AvroDataFrameReader(reader: DataFrameReader) {
|
||||
def avro: String => DataFrame = reader.format("com.uber.hoodie").load
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -18,9 +18,6 @@
|
||||
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
@@ -31,20 +28,21 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public class DataSourceTestUtils {
|
||||
|
||||
public static Optional<String> convertToString(HoodieRecord record) {
|
||||
try {
|
||||
String str = ((TestRawTripPayload) record.getData()).getJsonData();
|
||||
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
|
||||
return Optional.of(str.replaceAll("}", ", \"partition\": \"" + record.getPartitionPath() + "\"}"));
|
||||
} catch (IOException e) {
|
||||
return Optional.empty();
|
||||
}
|
||||
public static Optional<String> convertToString(HoodieRecord record) {
|
||||
try {
|
||||
String str = ((TestRawTripPayload) record.getData()).getJsonData();
|
||||
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
|
||||
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());
|
||||
}
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,7 +25,7 @@ import com.uber.hoodie.HoodieDataSourceHelpers;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -35,113 +35,123 @@ import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import java.util.List;
|
||||
/**
|
||||
* Sample program that writes & reads hoodie datasets via the Spark datasource
|
||||
*/
|
||||
public class HoodieJavaApp {
|
||||
|
||||
@Parameter(names={"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
|
||||
@Parameter(names={"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
||||
private String tableName = "hoodie_test";
|
||||
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
||||
private String tableName = "hoodie_test";
|
||||
|
||||
@Parameter(names={"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieJavaApp.class);
|
||||
private static Logger logger = LogManager.getLogger(HoodieJavaApp.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaApp cli = new HoodieJavaApp();
|
||||
JCommander cmd = new JCommander(cli, args);
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaApp cli = new HoodieJavaApp();
|
||||
JCommander cmd = new JCommander(cli, args);
|
||||
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
cli.run();
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
cli.run();
|
||||
}
|
||||
|
||||
public void run() throws Exception {
|
||||
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]")
|
||||
.getOrCreate();
|
||||
JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext());
|
||||
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
|
||||
// Spark session setup..
|
||||
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());
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
|
||||
/**
|
||||
* Commit with only inserts
|
||||
*/
|
||||
// Generate some input..
|
||||
List<String> records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001"/* ignore */, 100));
|
||||
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
|
||||
/**
|
||||
* Commit with only inserts
|
||||
*/
|
||||
// Generate some input..
|
||||
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
|
||||
.option("hoodie.insert.shuffle.parallelism", "2") // any hoodie client config can be passed like this
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2") // full list in HoodieWriteConfig & its package
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) // insert
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") // This is the record key
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") // this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "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
|
||||
.save(tablePath); // ultimately where the dataset will be placed
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("First commit at instant time :" + commitInstantTime1);
|
||||
// Save as hoodie dataset (copy on write)
|
||||
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",
|
||||
"2") // full list in HoodieWriteConfig & its package
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(),
|
||||
DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) // insert
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(),
|
||||
"_row_key") // This is the record key
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(),
|
||||
"partition") // this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(),
|
||||
"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
|
||||
.save(tablePath); // ultimately where the dataset will be placed
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("First commit at instant time :" + commitInstantTime1);
|
||||
|
||||
/**
|
||||
* Commit that updates records
|
||||
*/
|
||||
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")
|
||||
.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);
|
||||
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("Second commit at instant time :" + commitInstantTime1);
|
||||
/**
|
||||
* Commit that updates records
|
||||
*/
|
||||
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")
|
||||
.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);
|
||||
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
|
||||
.load(tablePath + "/*/*/*/*");
|
||||
hoodieROViewDF.registerTempTable("hoodie_ro");
|
||||
spark.sql("describe hoodie_ro").show();
|
||||
// all trips whose fare was greater than 2.
|
||||
spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 2.0").show();
|
||||
/**
|
||||
* Read & do some queries
|
||||
*/
|
||||
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();
|
||||
// all trips whose fare was greater than 2.
|
||||
spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 2.0")
|
||||
.show();
|
||||
|
||||
/**
|
||||
* Consume incrementally, only changes in commit 2 above.
|
||||
*/
|
||||
Dataset<Row> hoodieIncViewDF = spark.read().format("com.uber.hoodie")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(),
|
||||
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
|
||||
|
||||
/**
|
||||
* Consume incrementally, only changes in commit 2 above.
|
||||
*/
|
||||
Dataset<Row> hoodieIncViewDF = spark.read().format("com.uber.hoodie")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), 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
|
||||
|
||||
logger.info("You will only see records from : " + commitInstantTime2);
|
||||
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
logger.info("You will only see records from : " + commitInstantTime2);
|
||||
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,7 +20,6 @@ log4j.category.com.uber.hoodie.io=WARN
|
||||
log4j.category.com.uber.hoodie.common=WARN
|
||||
log4j.category.com.uber.hoodie.table.log=WARN
|
||||
log4j.category.org.apache.parquet.hadoop=WARN
|
||||
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
|
||||
@@ -16,9 +16,9 @@
|
||||
*
|
||||
*/
|
||||
|
||||
import com.uber.hoodie.{DataSourceWriteOptions, OverwriteWithLatestAvroPayload, SimpleKeyGenerator}
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import com.uber.hoodie.{DataSourceWriteOptions, OverwriteWithLatestAvroPayload, SimpleKeyGenerator}
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.commons.configuration.PropertiesConfiguration
|
||||
import org.junit.Assert._
|
||||
@@ -31,7 +31,7 @@ import org.scalatest.junit.AssertionsForJUnit
|
||||
class DataSourceDefaultsTest extends AssertionsForJUnit {
|
||||
|
||||
val schema = SchemaTestUtil.getComplexEvolvedSchema
|
||||
var baseRecord : GenericRecord = null
|
||||
var baseRecord: GenericRecord = null
|
||||
|
||||
@Before def initialize(): Unit = {
|
||||
baseRecord = SchemaTestUtil
|
||||
@@ -39,12 +39,13 @@ class DataSourceDefaultsTest extends AssertionsForJUnit {
|
||||
}
|
||||
|
||||
|
||||
private def getKeyConfig(recordKeyFieldName: String, paritionPathField: String): PropertiesConfiguration = {
|
||||
private def getKeyConfig(recordKeyFieldName: String, paritionPathField: String): PropertiesConfiguration = {
|
||||
val props = new PropertiesConfiguration()
|
||||
props.addProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
|
||||
props.addProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, paritionPathField)
|
||||
props
|
||||
}
|
||||
|
||||
@Test def testSimpleKeyGenerator() = {
|
||||
// top level, valid fields
|
||||
val hk1 = new SimpleKeyGenerator(getKeyConfig("field1", "name")).getKey(baseRecord)
|
||||
|
||||
@@ -20,11 +20,11 @@ import com.uber.hoodie.common.HoodieTestDataGenerator
|
||||
import com.uber.hoodie.common.util.FSUtils
|
||||
import com.uber.hoodie.config.HoodieWriteConfig
|
||||
import com.uber.hoodie.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hadoop.fs.FileSystem
|
||||
import org.apache.spark.sql._
|
||||
import org.junit.Assert._
|
||||
import org.junit.{Before, Test}
|
||||
import org.junit.rules.TemporaryFolder
|
||||
import org.junit.{Before, Test}
|
||||
import org.scalatest.junit.AssertionsForJUnit
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
@@ -44,8 +44,8 @@ class DataSourceTest extends AssertionsForJUnit {
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
var basePath : String = null
|
||||
var fs : FileSystem = null
|
||||
var basePath: String = null
|
||||
var fs: FileSystem = null
|
||||
|
||||
@Before def initialize() {
|
||||
spark = SparkSession.builder
|
||||
|
||||
Reference in New Issue
Block a user