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
@@ -25,10 +25,10 @@ import java.util.{Optional, Properties}
|
||||
import com.uber.hoodie.DataSourceReadOptions._
|
||||
import com.uber.hoodie.DataSourceWriteOptions._
|
||||
import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import com.uber.hoodie.common.util.TypedProperties
|
||||
import com.uber.hoodie.config.HoodieWriteConfig
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.commons.configuration.PropertiesConfiguration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
@@ -121,10 +121,10 @@ class DefaultSource extends RelationProvider
|
||||
mapAsScalaMap(defaultsMap)
|
||||
}
|
||||
|
||||
def toPropertiesConfiguration(params: Map[String, String]): PropertiesConfiguration = {
|
||||
val propCfg = new PropertiesConfiguration()
|
||||
params.foreach(kv => propCfg.addProperty(kv._1, kv._2))
|
||||
propCfg
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
params.foreach(kv => props.setProperty(kv._1, kv._2))
|
||||
props
|
||||
}
|
||||
|
||||
|
||||
@@ -161,7 +161,7 @@ class DefaultSource extends RelationProvider
|
||||
// Convert to RDD[HoodieRecord]
|
||||
val keyGenerator = DataSourceUtils.createKeyGenerator(
|
||||
parameters(KEYGENERATOR_CLASS_OPT_KEY),
|
||||
toPropertiesConfiguration(parameters)
|
||||
toProperties(parameters)
|
||||
)
|
||||
val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace)
|
||||
val hoodieRecords = genericRecords.map(gr => {
|
||||
|
||||
Reference in New Issue
Block a user