1
0

[HUDI-89] Add configOption & refactor all configs based on that (#2833)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-06-30 14:26:30 -07:00
committed by GitHub
parent 07e93de8b4
commit d412fb2fe6
173 changed files with 4277 additions and 3309 deletions

View File

@@ -71,7 +71,7 @@ public class HoodieDatasetBulkInsertHelper {
TypedProperties properties = new TypedProperties();
properties.putAll(config.getProps());
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY());
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key());
BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
StructType structTypeForUDF = rows.schema();

View File

@@ -71,10 +71,10 @@ class DefaultSource extends RelationProvider
optParams: Map[String, String],
schema: StructType): BaseRelation = {
// Add default options for unspecified read options keys.
val parameters = translateViewTypesToQueryTypes(optParams)
val parameters = DataSourceOptionsHelper.translateConfigurations(optParams)
val path = parameters.get("path")
val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY)
val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY.key)
if (path.isEmpty && readPathsStr.isEmpty) {
throw new HoodieException(s"'path' or '$READ_PATHS_OPT_KEY' or both must be specified.")
}
@@ -85,10 +85,10 @@ class DefaultSource extends RelationProvider
val fs = FSUtils.getFs(allPaths.head, sqlContext.sparkContext.hadoopConfiguration)
// Use the HoodieFileIndex only if the 'path' is not globbed.
// Or else we use the original way to read hoodie table.
val enableFileIndex = optParams.get(ENABLE_HOODIE_FILE_INDEX)
.map(_.toBoolean).getOrElse(DEFAULT_ENABLE_HOODIE_FILE_INDEX)
val enableFileIndex = optParams.get(ENABLE_HOODIE_FILE_INDEX.key)
.map(_.toBoolean).getOrElse(ENABLE_HOODIE_FILE_INDEX.defaultValue)
val useHoodieFileIndex = enableFileIndex && path.isDefined && !path.get.contains("*") &&
!parameters.contains(DataSourceReadOptions.READ_PATHS_OPT_KEY)
!parameters.contains(DataSourceReadOptions.READ_PATHS_OPT_KEY.key)
val globPaths = if (useHoodieFileIndex) {
None
} else {
@@ -105,7 +105,7 @@ class DefaultSource extends RelationProvider
val metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(tablePath).build()
val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent
val tableType = metaClient.getTableType
val queryType = parameters(QUERY_TYPE_OPT_KEY)
val queryType = parameters.getOrElse(QUERY_TYPE_OPT_KEY.key, QUERY_TYPE_OPT_KEY.defaultValue)
log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType")
(tableType, queryType, isBootstrappedTable) match {
@@ -157,7 +157,7 @@ class DefaultSource extends RelationProvider
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*)
if (translatedOptions(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
if (translatedOptions(OPERATION_OPT_KEY.key).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, translatedOptions, dfWithoutMetaCols)
} else {
HoodieSparkSqlWriter.write(sqlContext, mode, translatedOptions, dfWithoutMetaCols)

View File

@@ -28,17 +28,18 @@ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.client.HoodieWriteResult
import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties}
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP, DEFAULT_BOOTSTRAP_INDEX_CLASS}
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hive.util.ConfigUtils
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.internal.DataSourceInternalWriterHelper
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.sync.common.AbstractSyncTool
import org.apache.log4j.LogManager
import org.apache.spark.SPARK_VERSION
@@ -53,8 +54,6 @@ import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
import scala.collection.JavaConversions._
import scala.collection.mutable.ListBuffer
import org.apache.hudi.common.table.HoodieTableConfig.{DEFAULT_ARCHIVELOG_FOLDER}
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
object HoodieSparkSqlWriter {
@@ -75,22 +74,23 @@ object HoodieSparkSqlWriter {
val sparkContext = sqlContext.sparkContext
val path = parameters.get("path")
val tblNameOp = parameters.get(HoodieWriteConfig.TABLE_NAME)
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined
if (path.isEmpty || tblNameOp.isEmpty) {
throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.")
if (path.isEmpty) {
throw new HoodieException(s"'path' must be set.")
}
val tblName = tblNameOp.get.trim
val tblName = tblNameOp.trim
sparkContext.getConf.getOption("spark.serializer") match {
case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") =>
case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer")
}
val tableType = HoodieTableType.valueOf(parameters(TABLE_TYPE_OPT_KEY))
var operation = WriteOperationType.fromValue(parameters(OPERATION_OPT_KEY))
val tableType = HoodieTableType.valueOf(hoodieConfig.getString(TABLE_TYPE_OPT_KEY))
var operation = WriteOperationType.fromValue(hoodieConfig.getString(OPERATION_OPT_KEY))
// It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true
// Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly
// or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) .
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean &&
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS_OPT_KEY) &&
operation == WriteOperationType.UPSERT) {
log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " +
@@ -116,17 +116,15 @@ object HoodieSparkSqlWriter {
handleSaveModes(mode, basePath, tableConfig, tblName, operation, fs)
// Create the table if not present
if (!tableExists) {
val archiveLogFolder = parameters.getOrElse(
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER)
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
val partitionColumns = HoodieWriterUtils.getPartitionColumns(keyGenerator)
val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
.setTableType(tableType)
.setTableName(tblName)
.setArchiveLogFolder(archiveLogFolder)
.setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.setPartitionColumns(partitionColumns)
.initTable(sparkContext.hadoopConfiguration, path.get)
tableConfig = tableMetaClient.getTableConfig
@@ -136,7 +134,7 @@ object HoodieSparkSqlWriter {
// short-circuit if bulk_insert via row is enabled.
// scalastyle:off
if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean &&
if (hoodieConfig.getBoolean(ENABLE_ROW_WRITER_OPT_KEY) &&
operation == WriteOperationType.BULK_INSERT) {
val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName,
basePath, path, instantTime)
@@ -157,23 +155,23 @@ object HoodieSparkSqlWriter {
// Convert to RDD[HoodieRecord]
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, schema, structName, nameSpace)
val shouldCombine = parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean || operation.equals(WriteOperationType.UPSERT)
val shouldCombine = parameters(INSERT_DROP_DUPS_OPT_KEY.key()).toBoolean || operation.equals(WriteOperationType.UPSERT);
val hoodieAllIncomingRecords = genericRecords.map(gr => {
val hoodieRecord = if (shouldCombine) {
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false)
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, hoodieConfig.getString(PRECOMBINE_FIELD_OPT_KEY), false)
.asInstanceOf[Comparable[_]]
DataSourceUtils.createHoodieRecord(gr,
orderingVal, keyGenerator.getKey(gr),
parameters(PAYLOAD_CLASS_OPT_KEY))
hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
} else {
DataSourceUtils.createHoodieRecord(gr, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY))
DataSourceUtils.createHoodieRecord(gr, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
}
hoodieRecord
}).toJavaRDD()
// Create a HoodieWriteClient & issue the write.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP)
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key)
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
@@ -181,7 +179,7 @@ object HoodieSparkSqlWriter {
}
val hoodieRecords =
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS_OPT_KEY)) {
DataSourceUtils.dropDuplicates(jsc, hoodieAllIncomingRecords, mapAsJavaMap(parameters))
} else {
hoodieAllIncomingRecords
@@ -212,7 +210,7 @@ object HoodieSparkSqlWriter {
// Create a HoodieWriteClient & issue the delete.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
null, path.get, tblName,
mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP)))
mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key)))
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
@@ -260,13 +258,13 @@ object HoodieSparkSqlWriter {
val sparkContext = sqlContext.sparkContext
val path = parameters.getOrElse("path", throw new HoodieException("'path' must be set."))
val tableName = parameters.getOrElse(HoodieWriteConfig.TABLE_NAME,
throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}' must be set."))
val tableType = parameters(TABLE_TYPE_OPT_KEY)
val bootstrapBasePath = parameters.getOrElse(BOOTSTRAP_BASE_PATH_PROP,
throw new HoodieException(s"'${BOOTSTRAP_BASE_PATH_PROP}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
" operation'"))
val bootstrapIndexClass = parameters.getOrDefault(BOOTSTRAP_INDEX_CLASS_PROP, DEFAULT_BOOTSTRAP_INDEX_CLASS)
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
val tableType = hoodieConfig.getStringOrDefault(TABLE_TYPE_OPT_KEY)
val bootstrapBasePath = hoodieConfig.getStringOrThrow(BOOTSTRAP_BASE_PATH_PROP,
s"'${BOOTSTRAP_BASE_PATH_PROP.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
" operation'")
val bootstrapIndexClass = hoodieConfig.getStringOrDefault(BOOTSTRAP_INDEX_CLASS_PROP)
var schema: String = null
if (df.schema.nonEmpty) {
@@ -290,15 +288,14 @@ object HoodieSparkSqlWriter {
}
if (!tableExists) {
val archiveLogFolder = parameters.getOrElse(
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER)
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters)
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(HoodieTableType.valueOf(tableType))
.setTableName(tableName)
.setArchiveLogFolder(archiveLogFolder)
.setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.setBootstrapIndexClass(bootstrapIndexClass)
.setBootstrapBasePath(bootstrapBasePath)
.setPartitionColumns(partitionColumns)
@@ -313,7 +310,7 @@ object HoodieSparkSqlWriter {
} finally {
writeClient.close()
}
val metaSyncSuccess = metaSync(sqlContext.sparkSession, parameters, basePath, df.schema)
val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
metaSyncSuccess
}
@@ -333,10 +330,10 @@ object HoodieSparkSqlWriter {
val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
sparkContext.getConf.registerAvroSchemas(schema)
log.info(s"Registered avro schema : ${schema.toString(true)}")
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
if (parameters(INSERT_DROP_DUPS_OPT_KEY.key).toBoolean) {
throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
}
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA, schema.toString)
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
if (SPARK_VERSION.startsWith("2.")) {
@@ -347,7 +344,7 @@ object HoodieSparkSqlWriter {
} else if (SPARK_VERSION.startsWith("3.")) {
hoodieDF.write.format("org.apache.hudi.spark3.internal")
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
.option(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL, hoodieDF.schema.toDDL)
.option(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key, hoodieDF.schema.toDDL)
.options(params)
.mode(SaveMode.Append)
.save()
@@ -355,11 +352,12 @@ object HoodieSparkSqlWriter {
throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
+ " To use row writer please switch to spark 2 or spark 3")
}
val hiveSyncEnabled = params.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
val metaSyncEnabled = params.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params)
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED_OPT_KEY).toBoolean
val metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED_OPT_KEY).toBoolean
val syncHiveSuccess =
if (hiveSyncEnabled || metaSyncEnabled) {
metaSync(sqlContext.sparkSession, parameters, basePath, df.schema)
metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
} else {
true
}
@@ -399,47 +397,44 @@ object HoodieSparkSqlWriter {
}
}
private def syncHive(basePath: Path, fs: FileSystem, parameters: Map[String, String]): Boolean = {
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, parameters)
private def syncHive(basePath: Path, fs: FileSystem, hoodieConfig: HoodieConfig): Boolean = {
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, hoodieConfig)
val hiveConf: HiveConf = new HiveConf()
hiveConf.addResource(fs.getConf)
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable()
true
}
private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = {
private def buildSyncConfig(basePath: Path, hoodieConfig: HoodieConfig): HiveSyncConfig = {
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
hiveSyncConfig.basePath = basePath.toString
hiveSyncConfig.baseFileFormat = parameters(HIVE_BASE_FILE_FORMAT_OPT_KEY)
hiveSyncConfig.baseFileFormat = hoodieConfig.getString(HIVE_BASE_FILE_FORMAT_OPT_KEY)
hiveSyncConfig.usePreApacheInputFormat =
parameters.get(HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY).exists(r => r.toBoolean)
hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY)
hiveSyncConfig.tableName = parameters(HIVE_TABLE_OPT_KEY)
hiveSyncConfig.hiveUser = parameters(HIVE_USER_OPT_KEY)
hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY)
hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY)
hiveSyncConfig.skipROSuffix = parameters.getOrElse(HIVE_SKIP_RO_SUFFIX,
DataSourceWriteOptions.DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL).toBoolean
hoodieConfig.getStringOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY).toBoolean
hiveSyncConfig.databaseName = hoodieConfig.getString(HIVE_DATABASE_OPT_KEY)
hiveSyncConfig.tableName = hoodieConfig.getString(HIVE_TABLE_OPT_KEY)
hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER_OPT_KEY)
hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS_OPT_KEY)
hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL_OPT_KEY)
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX,
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.defaultValue).toBoolean
hiveSyncConfig.partitionFields =
ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY)
hiveSyncConfig.useJdbc = parameters(HIVE_USE_JDBC_OPT_KEY).toBoolean
hiveSyncConfig.useFileListingFromMetadata = parameters(HoodieMetadataConfig.METADATA_ENABLE_PROP).toBoolean
hiveSyncConfig.verifyMetadataFileListing = parameters(HoodieMetadataConfig.METADATA_VALIDATE_PROP).toBoolean
hiveSyncConfig.ignoreExceptions = parameters.get(HIVE_IGNORE_EXCEPTIONS_OPT_KEY).exists(r => r.toBoolean)
hiveSyncConfig.supportTimestamp = parameters.get(HIVE_SUPPORT_TIMESTAMP).exists(r => r.toBoolean)
hiveSyncConfig.autoCreateDatabase = parameters.getOrElse(HIVE_AUTO_CREATE_DATABASE_OPT_KEY,
DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY).toBoolean
hiveSyncConfig.decodePartition = parameters.getOrElse(URL_ENCODE_PARTITIONING_OPT_KEY,
DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL).toBoolean
ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY)
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC_OPT_KEY)
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_ENABLE_PROP)
hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_VALIDATE_PROP)
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS_OPT_KEY).toBoolean
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP).toBoolean
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE_OPT_KEY).toBoolean
hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING_OPT_KEY).toBoolean
val syncAsDtaSourceTable = parameters.getOrElse(DataSourceWriteOptions.HIVE_SYNC_AS_DATA_SOURCE_TABLE,
DataSourceWriteOptions.DEFAULT_HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
val syncAsDtaSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
if (syncAsDtaSourceTable) {
hiveSyncConfig.tableProperties = parameters.getOrElse(HIVE_TABLE_PROPERTIES, null)
val serdePropText = createSqlTableSerdeProperties(parameters, basePath.toString)
hiveSyncConfig.tableProperties = hoodieConfig.getStringOrDefault(HIVE_TABLE_PROPERTIES, null)
val serdePropText = createSqlTableSerdeProperties(hoodieConfig, basePath.toString)
val serdeProp = ConfigUtils.toMap(serdePropText)
serdeProp.put(ConfigUtils.SPARK_QUERY_TYPE_KEY, DataSourceReadOptions.QUERY_TYPE_OPT_KEY)
serdeProp.put(ConfigUtils.SPARK_QUERY_TYPE_KEY, DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key)
serdeProp.put(ConfigUtils.SPARK_QUERY_AS_RO_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
serdeProp.put(ConfigUtils.SPARK_QUERY_AS_RT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
@@ -453,18 +448,18 @@ object HoodieSparkSqlWriter {
* Add Spark Sql related table properties to the HIVE_TABLE_PROPERTIES.
* @param sqlConf The spark sql conf.
* @param schema The schema to write to the table.
* @param parameters The origin parameters.
* @param hoodieConfig The HoodieConfig contains origin parameters.
* @return A new parameters added the HIVE_TABLE_PROPERTIES property.
*/
private def addSqlTableProperties(sqlConf: SQLConf, schema: StructType,
parameters: Map[String, String]): Map[String, String] = {
hoodieConfig: HoodieConfig): HoodieConfig = {
// Convert the schema and partition info used by spark sql to hive table properties.
// The following code refers to the spark code in
// https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
// Sync schema with meta fields
val schemaWithMetaFields = HoodieSqlUtils.addMetaFields(schema)
val partitionSet = parameters(HIVE_PARTITION_FIELDS_OPT_KEY)
val partitionSet = hoodieConfig.getString(HIVE_PARTITION_FIELDS_OPT_KEY)
.split(",").map(_.trim).filter(!_.isEmpty).toSet
val threshold = sqlConf.getConf(SCHEMA_STRING_LENGTH_THRESHOLD)
@@ -487,31 +482,32 @@ object HoodieSparkSqlWriter {
}
}
var sqlPropertyText = ConfigUtils.configToString(properties)
sqlPropertyText = if (parameters.containsKey(HIVE_TABLE_PROPERTIES)) {
sqlPropertyText + "\n" + parameters(HIVE_TABLE_PROPERTIES)
sqlPropertyText = if (hoodieConfig.contains(HIVE_TABLE_PROPERTIES)) {
sqlPropertyText + "\n" + hoodieConfig.getString(HIVE_TABLE_PROPERTIES)
} else {
sqlPropertyText
}
parameters + (HIVE_TABLE_PROPERTIES -> sqlPropertyText)
hoodieConfig.setValue(HIVE_TABLE_PROPERTIES, sqlPropertyText)
hoodieConfig
}
private def createSqlTableSerdeProperties(parameters: Map[String, String], basePath: String): String = {
private def createSqlTableSerdeProperties(hoodieConfig: HoodieConfig, basePath: String): String = {
val pathProp = s"path=$basePath"
if (parameters.containsKey(HIVE_TABLE_SERDE_PROPERTIES)) {
pathProp + "\n" + parameters(HIVE_TABLE_SERDE_PROPERTIES)
if (hoodieConfig.contains(HIVE_TABLE_SERDE_PROPERTIES)) {
pathProp + "\n" + hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES)
} else {
pathProp
}
}
private def metaSync(spark: SparkSession, parameters: Map[String, String], basePath: Path,
private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path,
schema: StructType): Boolean = {
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
var metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED_OPT_KEY).toBoolean
var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED_OPT_KEY).toBoolean
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
parameters(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
val newParameters = addSqlTableProperties(spark.sessionState.conf, schema, parameters)
val newHoodieConfig = addSqlTableProperties(spark.sessionState.conf, schema, hoodieConfig)
// for backward compatibility
if (hiveSyncEnabled) {
metaSyncEnabled = true
@@ -523,13 +519,13 @@ object HoodieSparkSqlWriter {
syncClientToolClassSet.foreach(impl => {
val syncSuccess = impl.trim match {
case "org.apache.hudi.hive.HiveSyncTool" => {
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
syncHive(basePath, fs, newParameters)
log.info("Syncing to Hive Metastore (URL: " + hoodieConfig.getString(HIVE_URL_OPT_KEY) + ")")
syncHive(basePath, fs, newHoodieConfig)
true
}
case _ => {
val properties = new Properties();
properties.putAll(newParameters)
val properties = new Properties()
properties.putAll(newHoodieConfig.getProps)
properties.put("basePath", basePath.toString)
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
syncHoodie.syncHoodieTable()
@@ -559,7 +555,7 @@ object HoodieSparkSqlWriter {
if(writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors).isEmpty()) {
log.info("Proceeding to commit the write.")
val metaMap = parameters.filter(kv =>
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY)))
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY.key)))
val commitSuccess =
client.commit(tableInstantInfo.instantTime, writeResult.getWriteStatuses,
common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))),
@@ -583,7 +579,8 @@ object HoodieSparkSqlWriter {
log.info(s"Compaction Scheduled is $compactionInstant")
val metaSyncSuccess = metaSync(spark, parameters, tableInstantInfo.basePath, schema)
val metaSyncSuccess = metaSync(spark, HoodieWriterUtils.convertMapToHoodieConfig(parameters),
tableInstantInfo.basePath, schema)
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
if (!asyncCompactionEnabled) {
@@ -613,7 +610,7 @@ object HoodieSparkSqlWriter {
parameters: Map[String, String], configuration: Configuration) : Boolean = {
log.info(s"Config.inlineCompactionEnabled ? ${client.getConfig.inlineCompactionEnabled}")
if (asyncCompactionTriggerFnDefined && !client.getConfig.inlineCompactionEnabled
&& parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY).exists(r => r.toBoolean)) {
&& parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY.key).exists(r => r.toBoolean)) {
tableConfig.getTableType == HoodieTableType.MERGE_ON_READ
} else {
false

View File

@@ -47,9 +47,9 @@ class HoodieStreamingSink(sqlContext: SQLContext,
private val log = LogManager.getLogger(classOf[HoodieStreamingSink])
private val retryCnt = options(DataSourceWriteOptions.STREAMING_RETRY_CNT_OPT_KEY).toInt
private val retryIntervalMs = options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY).toLong
private val ignoreFailedBatch = options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY).toBoolean
private val retryCnt = options(DataSourceWriteOptions.STREAMING_RETRY_CNT_OPT_KEY.key).toInt
private val retryIntervalMs = options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY.key).toLong
private val ignoreFailedBatch = options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key).toBoolean
private var isAsyncCompactorServiceShutdownAbnormally = false
@@ -101,7 +101,7 @@ class HoodieStreamingSink(sqlContext: SQLContext,
log.error(s"Micro batch id=$batchId threw following exception: ", e)
if (ignoreFailedBatch) {
log.info(s"Ignore the exception and move on streaming as per " +
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration")
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key} configuration")
Success((true, None, None))
} else {
if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...")
@@ -115,7 +115,7 @@ class HoodieStreamingSink(sqlContext: SQLContext,
}))
if (ignoreFailedBatch) {
log.info(s"Ignore the errors and move on streaming as per " +
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration")
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key} configuration")
Success((true, None, None))
} else {
if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...")

View File

@@ -17,14 +17,14 @@
package org.apache.hudi
import java.util.Properties
import scala.collection.JavaConverters._
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.config.{HoodieConfig, TypedProperties}
import scala.collection.JavaConversions.mapAsJavaMap
import scala.collection.JavaConverters.mapAsScalaMapConverter
import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE
import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
@@ -46,36 +46,36 @@ object HoodieWriterUtils {
* @return
*/
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL,
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
METADATA_ENABLE_PROP -> DEFAULT_METADATA_ENABLE.toString,
METADATA_VALIDATE_PROP -> DEFAULT_METADATA_VALIDATE.toString,
COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL,
INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL,
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL,
HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL,
HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL,
HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL,
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL,
HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL,
HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL,
ASYNC_COMPACT_ENABLE_OPT_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL,
ENABLE_ROW_WRITER_OPT_KEY -> DEFAULT_ENABLE_ROW_WRITER_OPT_VAL
) ++ translateStorageTypeToTableType(parameters)
Map(OPERATION_OPT_KEY.key -> OPERATION_OPT_KEY.defaultValue,
TABLE_TYPE_OPT_KEY.key -> TABLE_TYPE_OPT_KEY.defaultValue,
PRECOMBINE_FIELD_OPT_KEY.key -> PRECOMBINE_FIELD_OPT_KEY.defaultValue,
PAYLOAD_CLASS_OPT_KEY.key -> PAYLOAD_CLASS_OPT_KEY.defaultValue,
RECORDKEY_FIELD_OPT_KEY.key -> RECORDKEY_FIELD_OPT_KEY.defaultValue,
PARTITIONPATH_FIELD_OPT_KEY.key -> PARTITIONPATH_FIELD_OPT_KEY.defaultValue,
KEYGENERATOR_CLASS_OPT_KEY.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
METADATA_ENABLE_PROP.key -> METADATA_ENABLE_PROP.defaultValue.toString,
METADATA_VALIDATE_PROP.key -> METADATA_VALIDATE_PROP.defaultValue.toString,
COMMIT_METADATA_KEYPREFIX_OPT_KEY.key -> COMMIT_METADATA_KEYPREFIX_OPT_KEY.defaultValue,
INSERT_DROP_DUPS_OPT_KEY.key -> INSERT_DROP_DUPS_OPT_KEY.defaultValue,
STREAMING_RETRY_CNT_OPT_KEY.key -> STREAMING_RETRY_CNT_OPT_KEY.defaultValue,
STREAMING_RETRY_INTERVAL_MS_OPT_KEY.key -> STREAMING_RETRY_INTERVAL_MS_OPT_KEY.defaultValue,
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key -> STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.defaultValue,
META_SYNC_CLIENT_TOOL_CLASS.key -> META_SYNC_CLIENT_TOOL_CLASS.defaultValue,
HIVE_SYNC_ENABLED_OPT_KEY.key -> HIVE_SYNC_ENABLED_OPT_KEY.defaultValue,
META_SYNC_ENABLED_OPT_KEY.key -> META_SYNC_ENABLED_OPT_KEY.defaultValue,
HIVE_DATABASE_OPT_KEY.key -> HIVE_DATABASE_OPT_KEY.defaultValue,
HIVE_TABLE_OPT_KEY.key -> HIVE_TABLE_OPT_KEY.defaultValue,
HIVE_BASE_FILE_FORMAT_OPT_KEY.key -> HIVE_BASE_FILE_FORMAT_OPT_KEY.defaultValue,
HIVE_USER_OPT_KEY.key -> HIVE_USER_OPT_KEY.defaultValue,
HIVE_PASS_OPT_KEY.key -> HIVE_PASS_OPT_KEY.defaultValue,
HIVE_URL_OPT_KEY.key -> HIVE_URL_OPT_KEY.defaultValue,
HIVE_PARTITION_FIELDS_OPT_KEY.key -> HIVE_PARTITION_FIELDS_OPT_KEY.defaultValue,
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.defaultValue,
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue,
HIVE_USE_JDBC_OPT_KEY.key -> HIVE_USE_JDBC_OPT_KEY.defaultValue,
ASYNC_COMPACT_ENABLE_OPT_KEY.key -> ASYNC_COMPACT_ENABLE_OPT_KEY.defaultValue,
ENABLE_ROW_WRITER_OPT_KEY.key -> ENABLE_ROW_WRITER_OPT_KEY.defaultValue
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
}
def toProperties(params: Map[String, String]): TypedProperties = {
@@ -111,4 +111,10 @@ object HoodieWriterUtils {
case _=> null
}
}
def convertMapToHoodieConfig(parameters: Map[String, String]): HoodieConfig = {
val properties = new Properties()
properties.putAll(mapAsJavaMap(parameters))
new HoodieConfig(properties)
}
}

View File

@@ -58,19 +58,19 @@ class IncrementalRelation(val sqlContext: SQLContext,
if (commitTimeline.empty()) {
throw new HoodieException("No instants to incrementally pull")
}
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key)) {
throw new HoodieException(s"Specify the begin instant time to pull from using " +
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key}")
}
val useEndInstantSchema = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY,
DataSourceReadOptions.DEFAULT_INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_VAL).toBoolean
val useEndInstantSchema = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY.key,
DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY.defaultValue).toBoolean
private val lastInstant = commitTimeline.lastInstant().get()
private val commitsToReturn = commitTimeline.findInstantsInRange(
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp))
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key),
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, lastInstant.getTimestamp))
.getInstants.iterator().toList
// use schema from a file produced in the end/latest instant
@@ -87,8 +87,8 @@ class IncrementalRelation(val sqlContext: SQLContext,
StructType(skeletonSchema.fields ++ dataSchema.fields)
}
private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY,
DataSourceReadOptions.DEFAULT_PUSH_DOWN_FILTERS_OPT_VAL).split(",").filter(!_.isEmpty)
private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY.key,
DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY.defaultValue).split(",").filter(!_.isEmpty)
override def schema: StructType = usedSchema
@@ -114,10 +114,10 @@ class IncrementalRelation(val sqlContext: SQLContext,
}
val pathGlobPattern = optParams.getOrElse(
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY,
DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.key,
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)
val (filteredRegularFullPaths, filteredMetaBootstrapFullPaths) = {
if(!pathGlobPattern.equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) {
if(!pathGlobPattern.equals(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)) {
val globMatcher = new GlobPattern("*" + pathGlobPattern)
(regularFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values,
metaBootstrapFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values)
@@ -140,7 +140,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
df = sqlContext.sparkSession.read
.format("hudi")
.schema(usedSchema)
.option(DataSourceReadOptions.READ_PATHS_OPT_KEY, filteredMetaBootstrapFullPaths.mkString(","))
.option(DataSourceReadOptions.READ_PATHS_OPT_KEY.key, filteredMetaBootstrapFullPaths.mkString(","))
.load()
}

View File

@@ -57,19 +57,19 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
if (commitTimeline.empty()) {
throw new HoodieException("No instants to incrementally pull")
}
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key)) {
throw new HoodieException(s"Specify the begin instant time to pull from using " +
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key}")
}
private val lastInstant = commitTimeline.lastInstant().get()
private val mergeType = optParams.getOrElse(
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY,
DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL)
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key,
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.defaultValue)
private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp))
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key),
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, lastInstant.getTimestamp))
log.debug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => f.toString).mkString(",")}")
private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
private val schemaUtil = new TableSchemaResolver(metaClient)
@@ -84,7 +84,7 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
} else {
// get preCombineFiled from the options if this is a old table which have not store
// the field to hoodie.properties
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD)
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
}
}
override def schema: StructType = tableStructSchema
@@ -178,10 +178,10 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
// Filter files based on user defined glob pattern
val pathGlobPattern = optParams.getOrElse(
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY,
DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.key,
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)
val filteredFileGroup = if(!pathGlobPattern
.equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) {
.equals(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)) {
val globMatcher = new GlobPattern("*" + pathGlobPattern)
fileGroup.filter(f => {
if (f.getLatestFileSlice.get().getBaseFile.isPresent) {

View File

@@ -74,8 +74,8 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
private lazy val tableStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
private val mergeType = optParams.getOrElse(
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY,
DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL)
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key,
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.defaultValue)
private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf)
private val preCombineField = {
val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField
@@ -84,7 +84,7 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
} else {
// get preCombineFiled from the options if this is a old table which have not store
// the field to hoodie.properties
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD)
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
}
}
override def schema: StructType = tableStructSchema

View File

@@ -41,27 +41,27 @@ object HoodieOptionConfig {
val SQL_KEY_TABLE_PRIMARY_KEY: HoodieOption[String] = buildConf()
.withSqlKey("primaryKey")
.withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)
.withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key)
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key)
.build()
val SQL_KEY_TABLE_TYPE: HoodieOption[String] = buildConf()
.withSqlKey("type")
.withHoodieKey(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY)
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME)
.withHoodieKey(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key)
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP.key)
.defaultValue(SQL_VALUE_TABLE_TYPE_COW)
.build()
val SQL_KEY_PRECOMBINE_FIELD: HoodieOption[String] = buildConf()
.withSqlKey("preCombineField")
.withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY)
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD)
.withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key)
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key)
.build()
val SQL_PAYLOAD_CLASS: HoodieOption[String] = buildConf()
.withSqlKey("payloadClass")
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY)
.withTableConfigKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME)
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY.key)
.withTableConfigKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP.key)
.defaultValue(classOf[DefaultHoodieRecordPayload].getName)
.build()
@@ -151,7 +151,7 @@ object HoodieOptionConfig {
*/
def getPrimaryColumns(options: Map[String, String]): Array[String] = {
val params = mappingSqlOptionToHoodieParam(options)
params.get(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
params.get(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key)
.map(_.split(",").filter(_.length > 0))
.getOrElse(Array.empty)
}
@@ -163,13 +163,13 @@ object HoodieOptionConfig {
*/
def getTableType(options: Map[String, String]): String = {
val params = mappingSqlOptionToHoodieParam(options)
params.getOrElse(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY,
DataSourceWriteOptions.DEFAULT_TABLE_TYPE_OPT_VAL)
params.getOrElse(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.defaultValue)
}
def getPreCombineField(options: Map[String, String]): Option[String] = {
val params = mappingSqlOptionToHoodieParam(options)
params.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY)
params.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key)
}
def buildConf[T](): HoodieOptions[T] = {

View File

@@ -44,7 +44,7 @@ class AlterHoodieTableRenameCommand(
.setConf(hadoopConf).build()
// Init table with new name.
HoodieTableMetaClient.withPropertyBuilder()
.fromProperties(metaClient.getTableConfig.getProperties)
.fromProperties(metaClient.getTableConfig.getProps)
.setTableName(newName.table)
.initTable(hadoopConf, path)
// Call AlterTableRenameCommand#run to rename table in meta.

View File

@@ -70,13 +70,13 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab
withSparkConf(sparkSession, targetTable.storage.properties) {
Map(
"path" -> path.toString,
KEYGENERATOR_CLASS_OPT_KEY -> classOf[SqlKeyGenerator].getCanonicalName,
TABLE_NAME -> tableId.table,
OPERATION_OPT_KEY -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
HIVE_SUPPORT_TIMESTAMP -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
HoodieWriteConfig.DELETE_PARALLELISM -> "200",
KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SqlKeyGenerator].getCanonicalName,
TABLE_NAME.key -> tableId.table,
OPERATION_OPT_KEY.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
HIVE_SUPPORT_TIMESTAMP.key -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
HoodieWriteConfig.DELETE_PARALLELISM.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
)
}

View File

@@ -189,7 +189,7 @@ object InsertIntoHoodieTableCommand {
}
val parameters = HoodieOptionConfig.mappingSqlOptionToHoodieParam(table.storage.properties)
val tableType = parameters.getOrElse(TABLE_TYPE_OPT_KEY, DEFAULT_TABLE_TYPE_OPT_VAL)
val tableType = parameters.getOrElse(TABLE_TYPE_OPT_KEY.key, TABLE_TYPE_OPT_KEY.defaultValue)
val partitionFields = table.partitionColumnNames.mkString(",")
val path = getTableLocation(table, sparkSession)
@@ -206,8 +206,8 @@ object InsertIntoHoodieTableCommand {
}
val dropDuplicate = sparkSession.conf
.getOption(INSERT_DROP_DUPS_OPT_KEY)
.getOrElse(DEFAULT_INSERT_DROP_DUPS_OPT_VAL)
.getOption(INSERT_DROP_DUPS_OPT_KEY.key)
.getOrElse(INSERT_DROP_DUPS_OPT_KEY.defaultValue)
.toBoolean
val operation = if (isOverwrite) {
@@ -236,25 +236,25 @@ object InsertIntoHoodieTableCommand {
withSparkConf(sparkSession, options) {
Map(
"path" -> path,
TABLE_TYPE_OPT_KEY -> tableType,
TABLE_NAME -> table.identifier.table,
PRECOMBINE_FIELD_OPT_KEY -> tableSchema.fields.last.name,
OPERATION_OPT_KEY -> operation,
KEYGENERATOR_CLASS_OPT_KEY -> keyGenClass,
RECORDKEY_FIELD_OPT_KEY -> primaryColumns.mkString(","),
PARTITIONPATH_FIELD_OPT_KEY -> partitionFields,
PAYLOAD_CLASS_OPT_KEY -> payloadClassName,
META_SYNC_ENABLED_OPT_KEY -> "true",
HIVE_USE_JDBC_OPT_KEY -> "false",
HIVE_DATABASE_OPT_KEY -> table.identifier.database.getOrElse("default"),
HIVE_TABLE_OPT_KEY -> table.identifier.table,
HIVE_SUPPORT_TIMESTAMP -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
HIVE_PARTITION_FIELDS_OPT_KEY -> partitionFields,
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING_OPT_KEY -> "true",
HoodieWriteConfig.INSERT_PARALLELISM -> "200",
HoodieWriteConfig.UPSERT_PARALLELISM -> "200",
TABLE_TYPE_OPT_KEY.key -> tableType,
TABLE_NAME.key -> table.identifier.table,
PRECOMBINE_FIELD_OPT_KEY.key -> tableSchema.fields.last.name,
OPERATION_OPT_KEY.key -> operation,
KEYGENERATOR_CLASS_OPT_KEY.key -> keyGenClass,
RECORDKEY_FIELD_OPT_KEY.key -> primaryColumns.mkString(","),
PARTITIONPATH_FIELD_OPT_KEY.key -> partitionFields,
PAYLOAD_CLASS_OPT_KEY.key -> payloadClassName,
META_SYNC_ENABLED_OPT_KEY.key -> "true",
HIVE_USE_JDBC_OPT_KEY.key -> "false",
HIVE_DATABASE_OPT_KEY.key -> table.identifier.database.getOrElse("default"),
HIVE_TABLE_OPT_KEY.key -> table.identifier.table,
HIVE_SUPPORT_TIMESTAMP.key -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
HIVE_PARTITION_FIELDS_OPT_KEY.key -> partitionFields,
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING_OPT_KEY.key -> "true",
HoodieWriteConfig.INSERT_PARALLELISM.key -> "200",
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL
)
}

View File

@@ -241,9 +241,9 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
// may be different from the target table, because the are transform logical in the update or
// insert actions.
var writeParams = parameters +
(OPERATION_OPT_KEY -> UPSERT_OPERATION_OPT_VAL) +
(HoodieWriteConfig.WRITE_SCHEMA_PROP -> getTableSchema.toString) +
(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> targetTableType)
(OPERATION_OPT_KEY.key -> UPSERT_OPERATION_OPT_VAL) +
(HoodieWriteConfig.WRITE_SCHEMA_PROP.key -> getTableSchema.toString) +
(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> targetTableType)
// Map of Condition -> Assignments
val updateConditionToAssignments =
@@ -288,8 +288,8 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
checkInsertAssignments(insertActions)
var writeParams = parameters +
(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL) +
(HoodieWriteConfig.WRITE_SCHEMA_PROP -> getTableSchema.toString)
(OPERATION_OPT_KEY.key -> INSERT_OPERATION_OPT_VAL) +
(HoodieWriteConfig.WRITE_SCHEMA_PROP.key -> getTableSchema.toString)
writeParams += (PAYLOAD_INSERT_CONDITION_AND_ASSIGNMENTS ->
serializedInsertConditionAndExpressions(insertActions))
@@ -431,24 +431,24 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
withSparkConf(sparkSession, options) {
Map(
"path" -> path,
RECORDKEY_FIELD_OPT_KEY -> targetKey2SourceExpression.keySet.mkString(","),
KEYGENERATOR_CLASS_OPT_KEY -> classOf[SqlKeyGenerator].getCanonicalName,
PRECOMBINE_FIELD_OPT_KEY -> targetKey2SourceExpression.keySet.head, // set a default preCombine field
TABLE_NAME -> targetTableName,
PARTITIONPATH_FIELD_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
PAYLOAD_CLASS_OPT_KEY -> classOf[ExpressionPayload].getCanonicalName,
META_SYNC_ENABLED_OPT_KEY -> "true",
HIVE_USE_JDBC_OPT_KEY -> "false",
HIVE_DATABASE_OPT_KEY -> targetTableDb,
HIVE_TABLE_OPT_KEY -> targetTableName,
HIVE_SUPPORT_TIMESTAMP -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
HIVE_PARTITION_FIELDS_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING_OPT_KEY -> "true", // enable the url decode for sql.
HoodieWriteConfig.INSERT_PARALLELISM -> "200", // set the default parallelism to 200 for sql
HoodieWriteConfig.UPSERT_PARALLELISM -> "200",
HoodieWriteConfig.DELETE_PARALLELISM -> "200",
RECORDKEY_FIELD_OPT_KEY.key -> targetKey2SourceExpression.keySet.mkString(","),
KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SqlKeyGenerator].getCanonicalName,
PRECOMBINE_FIELD_OPT_KEY.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field
TABLE_NAME.key -> targetTableName,
PARTITIONPATH_FIELD_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
PAYLOAD_CLASS_OPT_KEY.key -> classOf[ExpressionPayload].getCanonicalName,
META_SYNC_ENABLED_OPT_KEY.key -> "true",
HIVE_USE_JDBC_OPT_KEY.key -> "false",
HIVE_DATABASE_OPT_KEY.key -> targetTableDb,
HIVE_TABLE_OPT_KEY.key -> targetTableName,
HIVE_SUPPORT_TIMESTAMP.key -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
HIVE_PARTITION_FIELDS_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING_OPT_KEY.key -> "true", // enable the url decode for sql.
HoodieWriteConfig.INSERT_PARALLELISM.key -> "200", // set the default parallelism to 200 for sql
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
HoodieWriteConfig.DELETE_PARALLELISM.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
)
})

View File

@@ -46,7 +46,7 @@ class TruncateHoodieTableCommand(
// Create MetaClient
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
.setConf(hadoopConf).build()
Some(metaClient.getTableConfig.getProperties)
Some(metaClient.getTableConfig.getProps)
} else {
None
}

View File

@@ -96,22 +96,22 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
withSparkConf(sparkSession, targetTable.storage.properties) {
Map(
"path" -> path.toString,
RECORDKEY_FIELD_OPT_KEY -> primaryColumns.mkString(","),
KEYGENERATOR_CLASS_OPT_KEY -> classOf[SqlKeyGenerator].getCanonicalName,
PRECOMBINE_FIELD_OPT_KEY -> primaryColumns.head, //set the default preCombine field.
TABLE_NAME -> tableId.table,
OPERATION_OPT_KEY -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
META_SYNC_ENABLED_OPT_KEY -> "false", // TODO make the meta sync enable by default.
HIVE_USE_JDBC_OPT_KEY -> "false",
HIVE_DATABASE_OPT_KEY -> tableId.database.getOrElse("default"),
HIVE_TABLE_OPT_KEY -> tableId.table,
HIVE_PARTITION_FIELDS_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING_OPT_KEY -> "true",
HIVE_SUPPORT_TIMESTAMP -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
HoodieWriteConfig.UPSERT_PARALLELISM -> "200",
RECORDKEY_FIELD_OPT_KEY.key -> primaryColumns.mkString(","),
KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SqlKeyGenerator].getCanonicalName,
PRECOMBINE_FIELD_OPT_KEY.key -> primaryColumns.head, //set the default preCombine field.
TABLE_NAME.key -> tableId.table,
OPERATION_OPT_KEY.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
META_SYNC_ENABLED_OPT_KEY.key -> "false", // TODO make the meta sync enable by default.
HIVE_USE_JDBC_OPT_KEY.key -> "false",
HIVE_DATABASE_OPT_KEY.key -> tableId.database.getOrElse("default"),
HIVE_TABLE_OPT_KEY.key -> tableId.table,
HIVE_PARTITION_FIELDS_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING_OPT_KEY.key -> "true",
HIVE_SUPPORT_TIMESTAMP.key -> "true",
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
)
}

View File

@@ -171,7 +171,7 @@ class ExpressionPayload(record: GenericRecord,
}
private def isMORTable(properties: Properties): Boolean = {
properties.getProperty(TABLE_TYPE_OPT_KEY, null) == MOR_TABLE_TYPE_OPT_VAL
properties.getProperty(TABLE_TYPE_OPT_KEY.key, null) == MOR_TABLE_TYPE_OPT_VAL
}
private def convertToRecord(values: Array[AnyRef], schema: Schema): IndexedRecord = {
@@ -188,9 +188,9 @@ class ExpressionPayload(record: GenericRecord,
*/
private def initWriteSchemaIfNeed(properties: Properties): Unit = {
if (writeSchema == null) {
ValidationUtils.checkArgument(properties.containsKey(HoodieWriteConfig.WRITE_SCHEMA_PROP),
s"Missing ${HoodieWriteConfig.WRITE_SCHEMA_PROP}")
writeSchema = new Schema.Parser().parse(properties.getProperty(HoodieWriteConfig.WRITE_SCHEMA_PROP))
ValidationUtils.checkArgument(properties.containsKey(HoodieWriteConfig.WRITE_SCHEMA_PROP.key),
s"Missing ${HoodieWriteConfig.WRITE_SCHEMA_PROP.key}")
writeSchema = new Schema.Parser().parse(properties.getProperty(HoodieWriteConfig.WRITE_SCHEMA_PROP.key))
}
}

View File

@@ -154,8 +154,8 @@ class HoodieStreamSource(
} else {
// Consume the data between (startCommitTime, endCommitTime]
val incParams = parameters ++ Map(
DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY -> startCommitTime(startOffset),
DataSourceReadOptions.END_INSTANTTIME_OPT_KEY -> endOffset.commitTime
DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key -> startCommitTime(startOffset),
DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key -> endOffset.commitTime
)
val rdd = tableType match {