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

@@ -134,11 +134,11 @@ public class DataSourceUtils {
public static HoodieWriteConfig createHoodieConfig(String schemaStr, String basePath,
String tblName, Map<String, String> parameters) {
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY()));
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY())
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key()));
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key())
.equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
// insert/bulk-insert combining to be true, if filtering for duplicates
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY()));
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY().key()));
HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder()
.withPath(basePath).withAutoCommit(false).combineInput(combineInserts, true);
if (schemaStr != null) {
@@ -148,9 +148,9 @@ public class DataSourceUtils {
return builder.forTable(tblName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY()))
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY().key()))
.withInlineCompaction(inlineCompact).build())
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY()))
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key()))
.build())
// override above with Hoodie configs specified as options.
.withProps(parameters).build();
@@ -228,37 +228,37 @@ public class DataSourceUtils {
}
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) {
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key()));
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.usePreApacheInputFormat =
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL());
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY().key(),
Boolean.parseBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY().defaultValue()));
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().key(),
DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().defaultValue());
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key());
hiveSyncConfig.baseFileFormat = baseFileFormat;
hiveSyncConfig.hiveUser =
props.getString(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), DataSourceWriteOptions.DEFAULT_HIVE_USER_OPT_VAL());
props.getString(DataSourceWriteOptions.HIVE_USER_OPT_KEY().key(), DataSourceWriteOptions.HIVE_USER_OPT_KEY().defaultValue());
hiveSyncConfig.hivePass =
props.getString(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_HIVE_PASS_OPT_VAL());
props.getString(DataSourceWriteOptions.HIVE_PASS_OPT_KEY().key(), DataSourceWriteOptions.HIVE_PASS_OPT_KEY().defaultValue());
hiveSyncConfig.jdbcUrl =
props.getString(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), DataSourceWriteOptions.DEFAULT_HIVE_URL_OPT_VAL());
props.getString(DataSourceWriteOptions.HIVE_URL_OPT_KEY().key(), DataSourceWriteOptions.HIVE_URL_OPT_KEY().defaultValue());
hiveSyncConfig.partitionFields =
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), ",", new ArrayList<>());
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), ",", new ArrayList<>());
hiveSyncConfig.partitionValueExtractorClass =
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
SlashEncodedDayPartitionValueExtractor.class.getName());
hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL()));
hiveSyncConfig.autoCreateDatabase = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY()));
hiveSyncConfig.ignoreExceptions = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY()));
hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX(),
DataSourceWriteOptions.DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL()));
hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP(),
DataSourceWriteOptions.DEFAULT_HIVE_SUPPORT_TIMESTAMP()));
hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY().key(),
DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY().defaultValue()));
hiveSyncConfig.autoCreateDatabase = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE_OPT_KEY().key(),
DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE_OPT_KEY().defaultValue()));
hiveSyncConfig.ignoreExceptions = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS_OPT_KEY().key(),
DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS_OPT_KEY().defaultValue()));
hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX().key(),
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX().defaultValue()));
hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP().key(),
DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP().defaultValue()));
return hiveSyncConfig;
}
}

View File

@@ -17,21 +17,21 @@
package org.apache.hudi
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
import org.apache.hudi.common.model.WriteOperationType
import org.apache.hudi.common.config.ConfigProperty
import org.apache.hudi.common.fs.ConsistencyGuardConfig
import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType}
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.hive.HiveSyncTool
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
import org.apache.hudi.keygen.{CustomKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.hive.{HiveSyncTool, SlashEncodedDayPartitionValueExtractor}
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
import org.apache.hudi.keygen.{CustomKeyGenerator, SimpleKeyGenerator}
import org.apache.log4j.LogManager
import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils}
/**
* List of options that can be passed to the Hoodie datasource,
* in addition to the hoodie client configs
*/
* List of options that can be passed to the Hoodie datasource,
* in addition to the hoodie client configs
*/
/**
* Options supported for reading hoodie tables.
@@ -41,34 +41,46 @@ object DataSourceReadOptions {
private val log = LogManager.getLogger(DataSourceReadOptions.getClass)
/**
* Whether data needs to be read, in
*
* 1) Snapshot mode (obtain latest view, based on row & columnar data)
* 2) incremental mode (new data since an instantTime)
* 3) Read Optimized mode (obtain latest view, based on columnar data)
*
* Default: snapshot
*/
val QUERY_TYPE_OPT_KEY = "hoodie.datasource.query.type"
* Whether data needs to be read, in
*
* 1) Snapshot mode (obtain latest view, based on row & columnar data)
* 2) incremental mode (new data since an instantTime)
* 3) Read Optimized mode (obtain latest view, based on columnar data)
*
* Default: snapshot
*/
val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
val QUERY_TYPE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.query.type")
.defaultValue(QUERY_TYPE_SNAPSHOT_OPT_VAL)
.withAlternatives("hoodie.datasource.view.type")
.withDocumentation("Whether data needs to be read, in incremental mode (new data since an instantTime) " +
"(or) Read Optimized mode (obtain latest view, based on columnar data) (or) Snapshot mode " +
"(obtain latest view, based on row & columnar data)")
/**
* For Snapshot query on merge on read table. Use this key to define the payload class.
*/
val REALTIME_MERGE_OPT_KEY = "hoodie.datasource.merge.type"
val REALTIME_SKIP_MERGE_OPT_VAL = "skip_merge"
val REALTIME_PAYLOAD_COMBINE_OPT_VAL = "payload_combine"
val DEFAULT_REALTIME_MERGE_OPT_VAL = REALTIME_PAYLOAD_COMBINE_OPT_VAL
val REALTIME_MERGE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.merge.type")
.defaultValue(REALTIME_PAYLOAD_COMBINE_OPT_VAL)
.withDocumentation("")
val READ_PATHS_OPT_KEY = "hoodie.datasource.read.paths"
val READ_PATHS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.paths")
.noDefaultValue()
.withDocumentation("")
val READ_PRE_COMBINE_FIELD = HoodieWriteConfig.PRECOMBINE_FIELD_PROP
val ENABLE_HOODIE_FILE_INDEX = "hoodie.file.index.enable"
val DEFAULT_ENABLE_HOODIE_FILE_INDEX = true
val ENABLE_HOODIE_FILE_INDEX: ConfigProperty[Boolean] = ConfigProperty
.key("hoodie.file.index.enable")
.defaultValue(true)
.withDocumentation("")
@Deprecated
val VIEW_TYPE_OPT_KEY = "hoodie.datasource.view.type"
@@ -82,67 +94,62 @@ object DataSourceReadOptions {
val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
/**
* This eases migration from old configs to new configs.
*/
def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
VIEW_TYPE_INCREMENTAL_OPT_VAL -> QUERY_TYPE_INCREMENTAL_OPT_VAL,
VIEW_TYPE_REALTIME_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL)
if (!optParams.contains(QUERY_TYPE_OPT_KEY)) {
if (optParams.contains(VIEW_TYPE_OPT_KEY)) {
log.warn(VIEW_TYPE_OPT_KEY + " is deprecated and will be removed in a later release. Please use " + QUERY_TYPE_OPT_KEY)
optParams ++ Map(QUERY_TYPE_OPT_KEY -> translation(optParams(VIEW_TYPE_OPT_KEY)))
} else {
optParams ++ Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL)
}
} else {
optParams
}
}
* Instant time to start incrementally pulling data from. The instanttime here need not
* necessarily correspond to an instant on the timeline. New data written with an
* `instant_time > BEGIN_INSTANTTIME` are fetched out. For e.g: '20170901080000' will get
* all new data written after Sep 1, 2017 08:00AM.
*
* Default: None (Mandatory in incremental mode)
*/
val BEGIN_INSTANTTIME_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.begin.instanttime")
.noDefaultValue()
.withDocumentation("Instant time to start incrementally pulling data from. The instanttime here need not necessarily " +
"correspond to an instant on the timeline. New data written with an instant_time > BEGIN_INSTANTTIME are fetched out. " +
"For e.g: 20170901080000 will get all new data written after Sep 1, 2017 08:00AM.")
/**
* Instant time to start incrementally pulling data from. The instanttime here need not
* necessarily correspond to an instant on the timeline. New data written with an
* `instant_time > BEGIN_INSTANTTIME` are fetched out. For e.g: '20170901080000' will get
* all new data written after Sep 1, 2017 08:00AM.
*
* Default: None (Mandatory in incremental mode)
*/
val BEGIN_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.begin.instanttime"
* Instant time to limit incrementally fetched data to. New data written with an
* `instant_time <= END_INSTANTTIME` are fetched out.
*
* Default: latest instant (i.e fetches all new data since begin instant time)
*
*/
val END_INSTANTTIME_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.end.instanttime")
.noDefaultValue()
.withDocumentation("Instant time to limit incrementally fetched data to. " +
"New data written with an instant_time <= END_INSTANTTIME are fetched out.")
/**
* Instant time to limit incrementally fetched data to. New data written with an
* `instant_time <= END_INSTANTTIME` are fetched out.
*
* Default: latest instant (i.e fetches all new data since begin instant time)
*
*/
val END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.end.instanttime"
* If use the end instant schema when incrementally fetched data to.
*
* Default: false (use latest instant schema)
*
*/
val INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.schema.use.end.instanttime")
.defaultValue("false")
.withDocumentation("Uses end instant schema when incrementally fetched data to. Default: users latest instant schema.")
/**
* If use the end instant schema when incrementally fetched data to.
*
* Default: false (use latest instant schema)
*
*/
val INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.schema.use.end.instanttime"
val DEFAULT_INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_VAL = "false"
/**
* For use-cases like DeltaStreamer which reads from Hoodie Incremental table and applies opaque map functions,
* filters appearing late in the sequence of transformations cannot be automatically pushed down.
* This option allows setting filters directly on Hoodie Source
*/
val PUSH_DOWN_INCR_FILTERS_OPT_KEY = "hoodie.datasource.read.incr.filters"
val DEFAULT_PUSH_DOWN_FILTERS_OPT_VAL = ""
* For use-cases like DeltaStreamer which reads from Hoodie Incremental table and applies opaque map functions,
* filters appearing late in the sequence of transformations cannot be automatically pushed down.
* This option allows setting filters directly on Hoodie Source
*/
val PUSH_DOWN_INCR_FILTERS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.incr.filters")
.defaultValue("")
.withDocumentation("")
/**
* For the use-cases like users only want to incremental pull from certain partitions instead of the full table.
* This option allows using glob pattern to directly filter on path.
*/
val INCR_PATH_GLOB_OPT_KEY = "hoodie.datasource.read.incr.path.glob"
val DEFAULT_INCR_PATH_GLOB_OPT_VAL = ""
val INCR_PATH_GLOB_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.incr.path.glob")
.defaultValue("")
.withDocumentation("")
}
/**
@@ -153,11 +160,10 @@ object DataSourceWriteOptions {
private val log = LogManager.getLogger(DataSourceWriteOptions.getClass)
/**
* The write operation, that this write should do
*
* Default: upsert()
*/
val OPERATION_OPT_KEY = "hoodie.datasource.write.operation"
* The write operation, that this write should do
*
* Default: upsert()
*/
val BULK_INSERT_OPERATION_OPT_VAL = WriteOperationType.BULK_INSERT.value
val INSERT_OPERATION_OPT_VAL = WriteOperationType.INSERT.value
val UPSERT_OPERATION_OPT_VAL = WriteOperationType.UPSERT.value
@@ -165,18 +171,26 @@ object DataSourceWriteOptions {
val BOOTSTRAP_OPERATION_OPT_VAL = WriteOperationType.BOOTSTRAP.value
val INSERT_OVERWRITE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE.value
val INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE_TABLE.value
val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL
val OPERATION_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.operation")
.defaultValue(UPSERT_OPERATION_OPT_VAL)
.withDocumentation("Whether to do upsert, insert or bulkinsert for the write operation. " +
"Use bulkinsert to load new data into a table, and there on use upsert/insert. " +
"bulk insert uses a disk based write path to scale to load large inputs without need to cache it.")
/**
* The table type for the underlying data, for this write.
* Note that this can't change across writes.
*
* Default: COPY_ON_WRITE
*/
val TABLE_TYPE_OPT_KEY = "hoodie.datasource.write.table.type"
* The table type for the underlying data, for this write.
* Note that this can't change across writes.
*
* Default: COPY_ON_WRITE
*/
val COW_TABLE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
val MOR_TABLE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
val DEFAULT_TABLE_TYPE_OPT_VAL = COW_TABLE_TYPE_OPT_VAL
val TABLE_TYPE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.table.type")
.defaultValue(COW_TABLE_TYPE_OPT_VAL)
.withAlternatives("hoodie.datasource.write.storage.type")
.withDocumentation("The table type for the underlying data, for this write. This cant change between writes.")
@Deprecated
val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type"
@@ -187,15 +201,6 @@ object DataSourceWriteOptions {
@Deprecated
val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
def translateStorageTypeToTableType(optParams: Map[String, String]) : Map[String, String] = {
if (optParams.contains(STORAGE_TYPE_OPT_KEY) && !optParams.contains(TABLE_TYPE_OPT_KEY)) {
log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a later release; Please use " + TABLE_TYPE_OPT_KEY)
optParams ++ Map(TABLE_TYPE_OPT_KEY -> optParams(STORAGE_TYPE_OPT_KEY))
} else {
optParams
}
}
/**
* Translate spark parameters to hudi parameters
*
@@ -209,7 +214,7 @@ object DataSourceWriteOptions {
val partitionColumns = optParams.get(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)
.map(SparkDataSourceUtils.decodePartitioningColumns)
.getOrElse(Nil)
val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY,
val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key(),
DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
val partitionPathField =
@@ -228,17 +233,20 @@ object DataSourceWriteOptions {
case _ =>
partitionColumns.mkString(",")
}
translatedOptParams = optParams ++ Map(PARTITIONPATH_FIELD_OPT_KEY -> partitionPathField)
translatedOptParams = optParams ++ Map(PARTITIONPATH_FIELD_OPT_KEY.key -> partitionPathField)
}
translatedOptParams
}
/**
* Hive table name, to register the table into.
*
* Default: None (mandatory)
*/
val TABLE_NAME_OPT_KEY = "hoodie.datasource.write.table.name"
* Hive table name, to register the table into.
*
* Default: None (mandatory)
*/
val TABLE_NAME_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.table.name")
.noDefaultValue()
.withDocumentation("Hive table name, to register the table into.")
/**
* Field used in preCombining before actual write. When two records have the same
@@ -246,14 +254,12 @@ object DataSourceWriteOptions {
* determined by Object.compareTo(..)
*/
val PRECOMBINE_FIELD_OPT_KEY = HoodieWriteConfig.PRECOMBINE_FIELD_PROP
val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = "ts"
/**
* Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.
* This will render any value set for `PRECOMBINE_FIELD_OPT_VAL` in-effective
*/
val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS
val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName
/**
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
@@ -262,14 +268,12 @@ object DataSourceWriteOptions {
*
*/
val RECORDKEY_FIELD_OPT_KEY = KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY
val DEFAULT_RECORDKEY_FIELD_OPT_VAL = "uuid"
/**
* Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`. Actual
* value obtained by invoking .toString()
*/
val PARTITIONPATH_FIELD_OPT_KEY = KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY
val DEFAULT_PARTITIONPATH_FIELD_OPT_VAL = "partitionpath"
/**
* Flag to indicate whether to use Hive style partitioning.
@@ -277,9 +281,8 @@ object DataSourceWriteOptions {
* By default false (the names of partition folders are only partition values)
*/
val HIVE_STYLE_PARTITIONING_OPT_KEY = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY
val DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = KeyGeneratorOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL
val URL_ENCODE_PARTITIONING_OPT_KEY = KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY
val DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL
/**
* Key generator class, that implements will extract the key out of incoming record
*
@@ -291,97 +294,230 @@ object DataSourceWriteOptions {
* When set to true, will perform write operations directly using the spark native `Row` representation.
* By default, false (will be enabled as default in a future release)
*/
val ENABLE_ROW_WRITER_OPT_KEY = "hoodie.datasource.write.row.writer.enable"
val DEFAULT_ENABLE_ROW_WRITER_OPT_VAL = "false"
val ENABLE_ROW_WRITER_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.row.writer.enable")
.defaultValue("false")
.withDocumentation("")
/**
* Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata.
* This is useful to store checkpointing information, in a consistent way with the hoodie timeline
*/
val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix"
val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_"
* Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata.
* This is useful to store checkpointing information, in a consistent way with the hoodie timeline
*/
val COMMIT_METADATA_KEYPREFIX_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.commitmeta.key.prefix")
.defaultValue("_")
.withDocumentation("Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata. " +
"This is useful to store checkpointing information, in a consistent way with the hudi timeline")
/**
* Flag to indicate whether to drop duplicates upon insert.
* By default insert will accept duplicates, to gain extra performance.
*/
val INSERT_DROP_DUPS_OPT_KEY = "hoodie.datasource.write.insert.drop.duplicates"
val DEFAULT_INSERT_DROP_DUPS_OPT_VAL = "false"
* Flag to indicate whether to drop duplicates upon insert.
* By default insert will accept duplicates, to gain extra performance.
*/
val INSERT_DROP_DUPS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.insert.drop.duplicates")
.defaultValue("false")
.withDocumentation("If set to true, filters out all duplicate records from incoming dataframe, during insert operations.")
/**
* Flag to indicate how many times streaming job should retry for a failed microbatch
* By default 3
*/
val STREAMING_RETRY_CNT_OPT_KEY = "hoodie.datasource.write.streaming.retry.count"
val DEFAULT_STREAMING_RETRY_CNT_OPT_VAL = "3"
* Flag to indicate how many times streaming job should retry for a failed microbatch
* By default 3
*/
val STREAMING_RETRY_CNT_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.streaming.retry.count")
.defaultValue("3")
.withDocumentation("")
/**
* Flag to indicate how long (by millisecond) before a retry should issued for failed microbatch
* By default 2000 and it will be doubled by every retry
*/
val STREAMING_RETRY_INTERVAL_MS_OPT_KEY = "hoodie.datasource.write.streaming.retry.interval.ms"
val DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL = "2000"
* Flag to indicate how long (by millisecond) before a retry should issued for failed microbatch
* By default 2000 and it will be doubled by every retry
*/
val STREAMING_RETRY_INTERVAL_MS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.streaming.retry.interval.ms")
.defaultValue("2000")
.withDocumentation("")
/**
* Flag to indicate whether to ignore any non exception error (e.g. writestatus error)
* within a streaming microbatch
* By default true (in favor of streaming progressing over data integrity)
*/
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = "hoodie.datasource.write.streaming.ignore.failed.batch"
val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = "true"
val META_SYNC_CLIENT_TOOL_CLASS = "hoodie.meta.sync.client.tool.class"
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = classOf[HiveSyncTool].getName
* Flag to indicate whether to ignore any non exception error (e.g. writestatus error)
* within a streaming microbatch
* By default true (in favor of streaming progressing over data integrity)
*/
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.streaming.ignore.failed.batch")
.defaultValue("true")
.withDocumentation("")
val META_SYNC_CLIENT_TOOL_CLASS: ConfigProperty[String] = ConfigProperty
.key("hoodie.meta.sync.client.tool.class")
.defaultValue(classOf[HiveSyncTool].getName)
.withDocumentation("")
// HIVE SYNC SPECIFIC CONFIGS
//NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
// NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
// unexpected issues with config getting reset
val HIVE_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.hive_sync.enable"
val META_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.meta.sync.enable"
val HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database"
val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table"
val HIVE_BASE_FILE_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.base_file_format"
val HIVE_USER_OPT_KEY = "hoodie.datasource.hive_sync.username"
val HIVE_PASS_OPT_KEY = "hoodie.datasource.hive_sync.password"
val HIVE_URL_OPT_KEY = "hoodie.datasource.hive_sync.jdbcurl"
val HIVE_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.hive_sync.partition_fields"
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.hive_sync.partition_extractor_class"
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = "hoodie.datasource.hive_sync.assume_date_partitioning"
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.use_pre_apache_input_format"
val HIVE_USE_JDBC_OPT_KEY = "hoodie.datasource.hive_sync.use_jdbc"
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.auto_create_database"
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = "hoodie.datasource.hive_sync.ignore_exceptions"
val HIVE_SKIP_RO_SUFFIX = "hoodie.datasource.hive_sync.skip_ro_suffix"
val HIVE_SUPPORT_TIMESTAMP = "hoodie.datasource.hive_sync.support_timestamp"
val HIVE_TABLE_PROPERTIES = "hoodie.datasource.hive_sync.table_properties"
val HIVE_TABLE_SERDE_PROPERTIES = "hoodie.datasource.hive_sync.serde_properties"
val HIVE_SYNC_AS_DATA_SOURCE_TABLE = "hoodie.datasource.hive_sync.sync_as_datasource"
// DEFAULT FOR HIVE SPECIFIC CONFIGS
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = "false"
val DEFAULT_HIVE_DATABASE_OPT_VAL = "default"
val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown"
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = "PARQUET"
val DEFAULT_HIVE_USER_OPT_VAL = "hive"
val DEFAULT_HIVE_PASS_OPT_VAL = "hive"
val DEFAULT_HIVE_URL_OPT_VAL = "jdbc:hive2://localhost:10000"
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = ""
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false"
val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false"
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = "true"
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = "true"
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = "false"
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = "false"
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = "false"
val DEFAULT_HIVE_SYNC_AS_DATA_SOURCE_TABLE = "true"
val HIVE_SYNC_ENABLED_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.enable")
.defaultValue("false")
.withDocumentation("When set to true, register/sync the table to Apache Hive metastore")
val META_SYNC_ENABLED_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.meta.sync.enable")
.defaultValue("false")
.withDocumentation("")
val HIVE_DATABASE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.database")
.defaultValue("default")
.withDocumentation("database to sync to")
val HIVE_TABLE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.table")
.defaultValue("unknown")
.withDocumentation("table to sync to")
val HIVE_BASE_FILE_FORMAT_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.base_file_format")
.defaultValue("PARQUET")
.withDocumentation("")
val HIVE_USER_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.username")
.defaultValue("hive")
.withDocumentation("hive user name to use")
val HIVE_PASS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.password")
.defaultValue("hive")
.withDocumentation("hive password to use")
val HIVE_URL_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.jdbcurl")
.defaultValue("jdbc:hive2://localhost:10000")
.withDocumentation("Hive metastore url")
val HIVE_PARTITION_FIELDS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.partition_fields")
.defaultValue("")
.withDocumentation("field in the table to use for determining hive partition columns.")
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.partition_extractor_class")
.defaultValue(classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName)
.withDocumentation("")
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.assume_date_partitioning")
.defaultValue("false")
.withDocumentation("Assume partitioning is yyyy/mm/dd")
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.use_pre_apache_input_format")
.defaultValue("false")
.withDocumentation("")
val HIVE_USE_JDBC_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.use_jdbc")
.defaultValue("true")
.withDocumentation("Use JDBC when hive synchronization is enabled")
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.auto_create_database")
.defaultValue("true")
.withDocumentation("Auto create hive database if does not exists")
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.ignore_exceptions")
.defaultValue("false")
.withDocumentation("")
val HIVE_SKIP_RO_SUFFIX: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.skip_ro_suffix")
.defaultValue("false")
.withDocumentation("Skip the _ro suffix for Read optimized table, when registering")
val HIVE_SUPPORT_TIMESTAMP: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.support_timestamp")
.defaultValue("false")
.withDocumentation("INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type. " +
"Disabled by default for backward compatibility.")
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.table_properties")
.noDefaultValue()
.withDocumentation("")
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.serde_properties")
.noDefaultValue()
.withDocumentation("")
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.sync_as_datasource")
.defaultValue("true")
.withDocumentation("")
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable"
val DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL = "true"
val ASYNC_COMPACT_ENABLE_OPT_KEY: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.compaction.async.enable")
.defaultValue("true")
.withDocumentation("")
// Avro Kafka Source configs
val KAFKA_AVRO_VALUE_DESERIALIZER = "hoodie.deltastreamer.source.kafka.value.deserializer.class"
// Schema to be used in custom kakfa deserializer
val KAFKA_AVRO_VALUE_DESERIALIZER_SCHEMA = "hoodie.deltastreamer.source.kafka.value.deserializer.schema"
val KAFKA_AVRO_VALUE_DESERIALIZER: ConfigProperty[String] = ConfigProperty
.key("hoodie.deltastreamer.source.kafka.value.deserializer.class")
.noDefaultValue()
.withDocumentation("")
val KAFKA_AVRO_VALUE_DESERIALIZER_SCHEMA: ConfigProperty[String] = ConfigProperty
.key("hoodie.deltastreamer.source.kafka.value.deserializer.schema")
.noDefaultValue()
.withDocumentation("")
}
object DataSourceOptionsHelper {
private val log = LogManager.getLogger(DataSourceOptionsHelper.getClass)
// put all the configs with alternatives here
val allConfigsWithAlternatives = List(
DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY,
HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP,
HoodieTableConfig.HOODIE_LOG_FILE_FORMAT_PROP
)
// put all the deprecated configs here
val allDeprecatedConfigs: Set[String] = Set(
ConsistencyGuardConfig.CONSISTENCY_CHECK_ENABLED_PROP.key
)
// maps the deprecated config name to its latest name
val allAlternatives: Map[String, String] = {
val alterMap = scala.collection.mutable.Map[String, String]()
allConfigsWithAlternatives.foreach(cfg => cfg.getAlternatives.foreach(alternative => alterMap(alternative) = cfg.key))
alterMap.toMap
}
val viewTypeValueMap: Map[String, String] = Map(
DataSourceReadOptions.VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
DataSourceReadOptions.VIEW_TYPE_REALTIME_OPT_VAL -> DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
def translateConfigurations(optParams: Map[String, String]): Map[String, String] = {
val translatedOpt = scala.collection.mutable.Map[String, String]() ++= optParams
optParams.keySet.foreach(opt => {
if (allAlternatives.contains(opt) && !optParams.contains(allAlternatives(opt))) {
log.warn(opt + " is deprecated and will be removed in a later release; Please use " + allAlternatives(opt))
if (opt == DataSourceReadOptions.VIEW_TYPE_OPT_KEY) {
// special handle for VIEW_TYPE_OPT_KEY, also need to translate its values
translatedOpt ++= Map(allAlternatives(opt) -> viewTypeValueMap(optParams(opt)))
} else {
translatedOpt ++= Map(allAlternatives(opt) -> optParams(opt))
}
}
if (allDeprecatedConfigs.contains(opt)) {
log.warn(opt + " is deprecated and should never be used anymore")
}
})
translatedOpt.toMap
}
}