1
0

[HUDI-161] Remove --key-generator-class CLI arg in HoodieDeltaStreamer and use key generator class specified in datasource properties. (#781)

This commit is contained in:
Yihua Guo
2019-07-12 13:45:49 -07:00
committed by vinoth chandar
parent 11c4121f73
commit 621c246fa9
5 changed files with 62 additions and 22 deletions

View File

@@ -90,10 +90,17 @@ public class DataSourceUtils {
}
/**
* Create a key generator class via reflection, passing in any configs needed
* Create a key generator class via reflection, passing in any configs needed.
*
* If the class name of key generator is configured through the properties file, i.e., {@code
* props}, use the corresponding key generator class; otherwise, use the default key generator
* class specified in {@code DataSourceWriteOptions}.
*/
public static KeyGenerator createKeyGenerator(String keyGeneratorClass,
TypedProperties props) throws IOException {
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
String keyGeneratorClass = props.getString(
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL()
);
try {
return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
} catch (Throwable e) {

View File

@@ -84,10 +84,7 @@ private[hoodie] object HoodieSparkSqlWriter {
log.info(s"Registered avro schema : ${schema.toString(true)}")
// Convert to RDD[HoodieRecord]
val keyGenerator = DataSourceUtils.createKeyGenerator(
parameters(KEYGENERATOR_CLASS_OPT_KEY),
toProperties(parameters)
)
val keyGenerator = DataSourceUtils.createKeyGenerator(toProperties(parameters))
val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace)
val hoodieAllIncomingRecords = genericRecords.map(gr => {
val orderingVal = DataSourceUtils.getNestedFieldValAsString(