Reworking the deltastreamer tool
- Standardize version of jackson - DFSPropertiesConfiguration replaces usage of commons PropertiesConfiguration - Remove dependency on ConstructorUtils - Throw error if ordering value is not present, during key generation - Switch to shade plugin for hoodie-utilities - Added support for consumption for Confluent avro kafka serdes - Support for Confluent schema registry - KafkaSource now deals with skews nicely, by doing round robin allocation of source limit across partitions - Added support for BULK_INSERT operations as well - Pass in the payload class config properly into HoodieWriteClient - Fix documentation based on new usage - Adding tests on deltastreamer, sources and all new util classes.
This commit is contained in:
committed by
vinoth chandar
parent
fb95dbdedb
commit
d58ddbd999
@@ -18,6 +18,7 @@
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
|
||||
@@ -44,5 +45,8 @@ public abstract class BaseAvroPayload implements Serializable {
|
||||
public BaseAvroPayload(GenericRecord record, Comparable orderingVal) {
|
||||
this.record = record;
|
||||
this.orderingVal = orderingVal;
|
||||
if (orderingVal == null) {
|
||||
throw new HoodieException("Ordering value is null for record: " + record);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,6 +21,8 @@ package com.uber.hoodie;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.common.util.TypedProperties;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -31,8 +33,6 @@ 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;
|
||||
|
||||
@@ -72,10 +72,9 @@ public class DataSourceUtils {
|
||||
* Create a key generator class via reflection, passing in any configs needed
|
||||
*/
|
||||
public static KeyGenerator createKeyGenerator(String keyGeneratorClass,
|
||||
PropertiesConfiguration cfg) throws IOException {
|
||||
TypedProperties props) throws IOException {
|
||||
try {
|
||||
return (KeyGenerator) ConstructorUtils
|
||||
.invokeConstructor(Class.forName(keyGeneratorClass), (Object) cfg);
|
||||
return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
|
||||
}
|
||||
@@ -87,17 +86,17 @@ 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) ReflectionUtils
|
||||
.loadClass(payloadClass, new Class<?>[]{GenericRecord.class, Comparable.class}, record, orderingVal);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not create payload for class: " + payloadClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void checkRequiredProperties(PropertiesConfiguration configuration,
|
||||
public static void checkRequiredProperties(TypedProperties props,
|
||||
List<String> checkPropNames) {
|
||||
checkPropNames.stream().forEach(prop -> {
|
||||
if (!configuration.containsKey(prop)) {
|
||||
if (!props.containsKey(prop)) {
|
||||
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
|
||||
}
|
||||
});
|
||||
|
||||
@@ -19,9 +19,9 @@
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.util.TypedProperties;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
|
||||
/**
|
||||
* Abstract class to extend for plugging in extraction of
|
||||
@@ -30,9 +30,9 @@ import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
*/
|
||||
public abstract class KeyGenerator implements Serializable {
|
||||
|
||||
protected transient PropertiesConfiguration config;
|
||||
protected transient TypedProperties config;
|
||||
|
||||
protected KeyGenerator(PropertiesConfiguration config) {
|
||||
protected KeyGenerator(TypedProperties config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
|
||||
@@ -19,9 +19,9 @@
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.util.TypedProperties;
|
||||
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
|
||||
@@ -29,14 +29,16 @@ import org.apache.commons.configuration.PropertiesConfiguration;
|
||||
*/
|
||||
public class SimpleKeyGenerator extends KeyGenerator {
|
||||
|
||||
private static final String DEFAULT_PARTITION_PATH = "default";
|
||||
|
||||
protected final String recordKeyField;
|
||||
|
||||
protected final String partitionPathField;
|
||||
|
||||
public SimpleKeyGenerator(PropertiesConfiguration config) {
|
||||
super(config);
|
||||
this.recordKeyField = config.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
this.partitionPathField = config
|
||||
public SimpleKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
|
||||
this.partitionPathField = props
|
||||
.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
|
||||
}
|
||||
|
||||
@@ -46,7 +48,16 @@ public class SimpleKeyGenerator extends KeyGenerator {
|
||||
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));
|
||||
|
||||
String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
|
||||
String partitionPath;
|
||||
try {
|
||||
partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField);
|
||||
} catch (HoodieException e) {
|
||||
// if field is not found, lump it into default partition
|
||||
partitionPath = DEFAULT_PARTITION_PATH;
|
||||
}
|
||||
|
||||
return new HoodieKey(recordKey, partitionPath);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user