1
0

[HUDI-2255] Refactor Datasource options (#3373)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-08-03 17:50:30 -07:00
committed by GitHub
parent 1ff2d3459a
commit 91bb0d1318
83 changed files with 887 additions and 887 deletions

View File

@@ -136,9 +136,9 @@ public class DataSourceUtils {
public static Map<String, String> getExtraMetadata(Map<String, String> properties) {
Map<String, String> extraMetadataMap = new HashMap<>();
if (properties.containsKey(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key())) {
if (properties.containsKey(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX().key())) {
properties.entrySet().forEach(entry -> {
if (entry.getKey().startsWith(properties.get(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key()))) {
if (entry.getKey().startsWith(properties.get(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX().key()))) {
extraMetadataMap.put(entry.getKey(), entry.getValue());
}
});
@@ -169,13 +169,13 @@ 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().key()));
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key())
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key()));
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE().key())
.equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
boolean asyncClusteringEnabled = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE_OPT_KEY().key()));
boolean inlineClusteringEnabled = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INLINE_CLUSTERING_ENABLE_OPT_KEY().key()));
boolean asyncClusteringEnabled = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key()));
boolean inlineClusteringEnabled = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INLINE_CLUSTERING_ENABLE().key()));
// insert/bulk-insert combining to be true, if filtering for duplicates
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY().key()));
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS().key()));
HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder()
.withPath(basePath).withAutoCommit(false).combineInput(combineInserts, true);
if (schemaStr != null) {
@@ -185,12 +185,12 @@ 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().key()))
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS().key()))
.withInlineCompaction(inlineCompact).build())
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
.withInlineClustering(inlineClusteringEnabled)
.withAsyncClustering(asyncClusteringEnabled).build())
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key()))
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD().key()))
.build())
// override above with Hoodie configs specified as options.
.withProps(parameters).build();
@@ -268,33 +268,33 @@ public class DataSourceUtils {
}
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) {
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key()));
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE().key()));
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.usePreApacheInputFormat =
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());
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT().key(),
Boolean.parseBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT().defaultValue()));
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE().key(),
DataSourceWriteOptions.HIVE_DATABASE().defaultValue());
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE().key());
hiveSyncConfig.baseFileFormat = baseFileFormat;
hiveSyncConfig.hiveUser =
props.getString(DataSourceWriteOptions.HIVE_USER_OPT_KEY().key(), DataSourceWriteOptions.HIVE_USER_OPT_KEY().defaultValue());
props.getString(DataSourceWriteOptions.HIVE_USER().key(), DataSourceWriteOptions.HIVE_USER().defaultValue());
hiveSyncConfig.hivePass =
props.getString(DataSourceWriteOptions.HIVE_PASS_OPT_KEY().key(), DataSourceWriteOptions.HIVE_PASS_OPT_KEY().defaultValue());
props.getString(DataSourceWriteOptions.HIVE_PASS().key(), DataSourceWriteOptions.HIVE_PASS().defaultValue());
hiveSyncConfig.jdbcUrl =
props.getString(DataSourceWriteOptions.HIVE_URL_OPT_KEY().key(), DataSourceWriteOptions.HIVE_URL_OPT_KEY().defaultValue());
props.getString(DataSourceWriteOptions.HIVE_URL().key(), DataSourceWriteOptions.HIVE_URL().defaultValue());
hiveSyncConfig.partitionFields =
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), ",", new ArrayList<>());
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), ",", new ArrayList<>());
hiveSyncConfig.partitionValueExtractorClass =
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
SlashEncodedDayPartitionValueExtractor.class.getName());
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.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC().key(),
DataSourceWriteOptions.HIVE_USE_JDBC().defaultValue()));
hiveSyncConfig.autoCreateDatabase = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE().key(),
DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE().defaultValue()));
hiveSyncConfig.ignoreExceptions = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().key(),
DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().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(),

View File

@@ -106,7 +106,7 @@ public class BulkInsertDataInternalWriterHelper {
private Option<BuiltinKeyGenerator> getKeyGenerator(Properties properties) {
TypedProperties typedProperties = new TypedProperties();
typedProperties.putAll(properties);
if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
return Option.empty(); // Do not instantiate NonPartitionKeyGen
} else {
try {

View File

@@ -17,7 +17,7 @@
package org.apache.hudi
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE_OPT_KEY, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.common.config.ConfigProperty
import org.apache.hudi.common.fs.ConsistencyGuardConfig
import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType}
@@ -45,7 +45,7 @@ object DataSourceReadOptions {
val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
val QUERY_TYPE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val QUERY_TYPE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.query.type")
.defaultValue(QUERY_TYPE_SNAPSHOT_OPT_VAL)
.withAlternatives("hoodie.datasource.view.type")
@@ -55,14 +55,14 @@ object DataSourceReadOptions {
val REALTIME_SKIP_MERGE_OPT_VAL = "skip_merge"
val REALTIME_PAYLOAD_COMBINE_OPT_VAL = "payload_combine"
val REALTIME_MERGE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val REALTIME_MERGE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.merge.type")
.defaultValue(REALTIME_PAYLOAD_COMBINE_OPT_VAL)
.withDocumentation("For Snapshot query on merge on read table, control whether we invoke the record " +
s"payload implementation to merge (${REALTIME_PAYLOAD_COMBINE_OPT_VAL}) or skip merging altogether" +
s"${REALTIME_SKIP_MERGE_OPT_VAL}")
val READ_PATHS_OPT_KEY: ConfigProperty[String] = ConfigProperty
val READ_PATHS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.paths")
.noDefaultValue()
.withDocumentation("Comma separated list of file paths to read within a Hudi table.")
@@ -86,32 +86,32 @@ object DataSourceReadOptions {
@Deprecated
val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
val BEGIN_INSTANTTIME_OPT_KEY: ConfigProperty[String] = ConfigProperty
val BEGIN_INSTANTTIME: 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.")
val END_INSTANTTIME_OPT_KEY: ConfigProperty[String] = ConfigProperty
val END_INSTANTTIME: 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.")
val INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY: ConfigProperty[String] = ConfigProperty
val INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME: 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.")
val PUSH_DOWN_INCR_FILTERS_OPT_KEY: ConfigProperty[String] = ConfigProperty
val PUSH_DOWN_INCR_FILTERS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.incr.filters")
.defaultValue("")
.withDocumentation("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 INCR_PATH_GLOB_OPT_KEY: ConfigProperty[String] = ConfigProperty
val INCR_PATH_GLOB: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.read.incr.path.glob")
.defaultValue("")
.withDocumentation("For the use-cases like users only want to incremental pull from certain partitions "
@@ -132,7 +132,7 @@ 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 OPERATION_OPT_KEY: ConfigProperty[String] = ConfigProperty
val OPERATION: 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. " +
@@ -141,14 +141,14 @@ object DataSourceWriteOptions {
val COW_TABLE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
val MOR_TABLE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
val TABLE_TYPE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val TABLE_TYPE: 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"
val STORAGE_TYPE_OPT = "hoodie.datasource.write.storage.type"
@Deprecated
val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
@Deprecated
@@ -164,12 +164,12 @@ object DataSourceWriteOptions {
*/
def translateSqlOptions(optParams: Map[String, String]): Map[String, String] = {
var translatedOptParams = optParams
// translate the api partitionBy of spark DataFrameWriter to PARTITIONPATH_FIELD_OPT_KEY
// translate the api partitionBy of spark DataFrameWriter to PARTITIONPATH_FIELD
if (optParams.contains(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)) {
val partitionColumns = optParams.get(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)
.map(SparkDataSourceUtils.decodePartitioningColumns)
.getOrElse(Nil)
val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key(),
val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS.key(),
DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
val partitionPathField =
@@ -188,12 +188,12 @@ object DataSourceWriteOptions {
case _ =>
partitionColumns.mkString(",")
}
translatedOptParams = optParams ++ Map(PARTITIONPATH_FIELD_OPT_KEY.key -> partitionPathField)
translatedOptParams = optParams ++ Map(PARTITIONPATH_FIELD.key -> partitionPathField)
}
translatedOptParams
}
val TABLE_NAME_OPT_KEY: ConfigProperty[String] = ConfigProperty
val TABLE_NAME: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.table.name")
.noDefaultValue()
.withDocumentation("Table name for the datasource write. Also used to register the table into meta stores.")
@@ -203,13 +203,13 @@ object DataSourceWriteOptions {
* key value, we will pick the one with the largest value for the precombine field,
* determined by Object.compareTo(..)
*/
val PRECOMBINE_FIELD_OPT_KEY = HoodieWriteConfig.PRECOMBINE_FIELD_PROP
val PRECOMBINE_FIELD = HoodieWriteConfig.PRECOMBINE_FIELD_PROP
/**
* 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 PAYLOAD_CLASS = HoodieWriteConfig.WRITE_PAYLOAD_CLASS
/**
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
@@ -217,56 +217,56 @@ object DataSourceWriteOptions {
* the dot notation eg: `a.b.c`
*
*/
val RECORDKEY_FIELD_OPT_KEY = KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY
val RECORDKEY_FIELD = KeyGeneratorOptions.RECORDKEY_FIELD
/**
* 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 PARTITIONPATH_FIELD = KeyGeneratorOptions.PARTITIONPATH_FIELD
/**
* Flag to indicate whether to use Hive style partitioning.
* If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.
* 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 URL_ENCODE_PARTITIONING_OPT_KEY = KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY
val HIVE_STYLE_PARTITIONING = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING
val URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING
/**
* Key generator class, that implements will extract the key out of incoming record
*
*/
val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS_PROP
val KEYGENERATOR_CLASS = HoodieWriteConfig.KEYGENERATOR_CLASS_PROP
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName
/**
*
* By default, false (will be enabled as default in a future release)
*/
val ENABLE_ROW_WRITER_OPT_KEY: ConfigProperty[String] = ConfigProperty
val ENABLE_ROW_WRITER: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.row.writer.enable")
.defaultValue("false")
.withDocumentation("When set to true, will perform write operations directly using the spark native " +
"`Row` representation, avoiding any additional conversion costs.")
val COMMIT_METADATA_KEYPREFIX_OPT_KEY: ConfigProperty[String] = ConfigProperty
val COMMIT_METADATA_KEYPREFIX: 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")
val INSERT_DROP_DUPS_OPT_KEY: ConfigProperty[String] = ConfigProperty
val INSERT_DROP_DUPS: 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.")
val STREAMING_RETRY_CNT_OPT_KEY: ConfigProperty[String] = ConfigProperty
val STREAMING_RETRY_CNT: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.streaming.retry.count")
.defaultValue("3")
.withDocumentation("Config to indicate how many times streaming job should retry for a failed micro batch.")
val STREAMING_RETRY_INTERVAL_MS_OPT_KEY: ConfigProperty[String] = ConfigProperty
val STREAMING_RETRY_INTERVAL_MS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.streaming.retry.interval.ms")
.defaultValue("2000")
.withDocumentation(" Config to indicate how long (by millisecond) before a retry should issued for failed microbatch")
@@ -275,7 +275,7 @@ object DataSourceWriteOptions {
*
* By default true (in favor of streaming progressing over data integrity)
*/
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY: ConfigProperty[String] = ConfigProperty
val STREAMING_IGNORE_FAILED_BATCH: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.streaming.ignore.failed.batch")
.defaultValue("true")
.withDocumentation("Config to indicate whether to ignore any non exception error (e.g. writestatus error)"
@@ -289,77 +289,77 @@ object DataSourceWriteOptions {
// HIVE SYNC SPECIFIC CONFIGS
// 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: ConfigProperty[String] = ConfigProperty
val HIVE_SYNC_ENABLED: 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
val META_SYNC_ENABLED: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.meta.sync.enable")
.defaultValue("false")
.withDocumentation("")
val HIVE_DATABASE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val HIVE_DATABASE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.database")
.defaultValue("default")
.withDocumentation("database to sync to")
val HIVE_TABLE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val HIVE_TABLE: 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
val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.base_file_format")
.defaultValue("PARQUET")
.withDocumentation("Base file format for the sync.")
val HIVE_USER_OPT_KEY: ConfigProperty[String] = ConfigProperty
val HIVE_USER: 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
val HIVE_PASS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.password")
.defaultValue("hive")
.withDocumentation("hive password to use")
val HIVE_URL_OPT_KEY: ConfigProperty[String] = ConfigProperty
val HIVE_URL: 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
val HIVE_PARTITION_FIELDS: 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
val HIVE_PARTITION_EXTRACTOR_CLASS: 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
val HIVE_ASSUME_DATE_PARTITION: 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
val HIVE_USE_PRE_APACHE_INPUT_FORMAT: 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
val HIVE_USE_JDBC: 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
val HIVE_AUTO_CREATE_DATABASE: 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
val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.ignore_exceptions")
.defaultValue("false")
.withDocumentation("")
@@ -402,18 +402,18 @@ object DataSourceWriteOptions {
.withDocumentation("The number of partitions one batch when synchronous partitions to hive.")
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.compaction.async.enable")
.defaultValue("true")
.withDocumentation("Controls whether async compaction should be turned on for MOR table writing.")
val INLINE_CLUSTERING_ENABLE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val INLINE_CLUSTERING_ENABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.clustering.inline.enable")
.defaultValue("false")
.sinceVersion("0.9.0")
.withDocumentation("Enable inline clustering. Disabled by default.")
val ASYNC_CLUSTERING_ENABLE_OPT_KEY: ConfigProperty[String] = ConfigProperty
val ASYNC_CLUSTERING_ENABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.clustering.async.enable")
.defaultValue("false")
.sinceVersion("0.9.0")
@@ -432,8 +432,8 @@ object DataSourceOptionsHelper {
// put all the configs with alternatives here
val allConfigsWithAlternatives = List(
DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY,
DataSourceReadOptions.QUERY_TYPE,
DataSourceWriteOptions.TABLE_TYPE,
HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP,
HoodieTableConfig.HOODIE_LOG_FILE_FORMAT_PROP
)
@@ -461,7 +461,7 @@ object DataSourceOptionsHelper {
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
// special handle for VIEW_TYPE, also need to translate its values
translatedOpt ++= Map(allAlternatives(opt) -> viewTypeValueMap(optParams(opt)))
} else {
translatedOpt ++= Map(allAlternatives(opt) -> optParams(opt))
@@ -476,13 +476,13 @@ object DataSourceOptionsHelper {
def parametersWithReadDefaults(parameters: Map[String, String]): Map[String, String] = {
// First check if the ConfigUtils.IS_QUERY_AS_RO_TABLE has set by HiveSyncTool,
// or else use query type from QUERY_TYPE_OPT_KEY.
// or else use query type from QUERY_TYPE.
val queryType = parameters.get(ConfigUtils.IS_QUERY_AS_RO_TABLE)
.map(is => if (is.toBoolean) QUERY_TYPE_READ_OPTIMIZED_OPT_VAL else QUERY_TYPE_SNAPSHOT_OPT_VAL)
.getOrElse(parameters.getOrElse(QUERY_TYPE_OPT_KEY.key, QUERY_TYPE_OPT_KEY.defaultValue()))
.getOrElse(parameters.getOrElse(QUERY_TYPE.key, QUERY_TYPE.defaultValue()))
Map(
QUERY_TYPE_OPT_KEY.key -> queryType
QUERY_TYPE.key -> queryType
) ++ translateConfigurations(parameters)
}
}

View File

@@ -72,9 +72,9 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
protected HoodieWriteConfig getWriteConfig(boolean populateMetaFields) {
Properties properties = new Properties();
if (!populateMetaFields) {
properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(), SimpleKeyGenerator.class.getName());
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), SparkDatasetTestUtils.RECORD_KEY_FIELD_NAME);
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), SimpleKeyGenerator.class.getName());
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key(), SparkDatasetTestUtils.RECORD_KEY_FIELD_NAME);
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
properties.setProperty(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
}
return getConfigBuilder(basePath).withProperties(properties).build();