1
0

[HUDI-2883] Refactor hive sync tool / config to use reflection and standardize configs (#4175)

- Refactor hive sync tool / config to use reflection and standardize configs

Co-authored-by: sivabalan <n.siva.b@gmail.com>
Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
Rajesh Mahindra
2022-03-21 19:56:31 -07:00
committed by GitHub
parent 9b6e138af2
commit 5f570ea151
43 changed files with 1521 additions and 1217 deletions

View File

@@ -42,6 +42,7 @@ import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
import org.apache.hudi.sync.common.HoodieSyncConfig;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.util.DataTypeUtils;
@@ -270,6 +271,11 @@ public class DataSourceUtils {
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
}
/**
* @deprecated Use {@link HiveSyncConfig} constructor directly and provide the props,
* and set {@link HoodieSyncConfig#META_SYNC_BASE_PATH} and {@link HoodieSyncConfig#META_SYNC_BASE_FILE_FORMAT} instead.
*/
@Deprecated
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) {
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE().key()));
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
@@ -310,7 +316,7 @@ public class DataSourceUtils {
hiveSyncConfig.isConditionalSync = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().key(),
DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().defaultValue()));
hiveSyncConfig.bucketSpec = props.getBoolean(DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().key(),
(boolean) DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue())
DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue())
? HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()),
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())) : null;
if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION())) {

View File

@@ -25,9 +25,10 @@ import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.common.util.Option
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.hive.util.ConfigUtils
import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor}
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.log4j.LogManager
import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils}
@@ -246,7 +247,7 @@ object DataSourceWriteOptions {
}
val TABLE_NAME: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.write.table.name")
.key(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)
.noDefaultValue()
.withDocumentation("Table name for the datasource write. Also used to register the table into meta stores.")
@@ -380,185 +381,79 @@ 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: 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: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.meta.sync.enable")
.defaultValue("false")
.withDocumentation("")
val HIVE_DATABASE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.database")
.defaultValue("default")
.withDocumentation("database to sync to")
val hiveTableOptKeyInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => {
if (p.contains(TABLE_NAME)) {
Option.of(p.getString(TABLE_NAME))
} else if (p.contains(HoodieWriteConfig.TBL_NAME)) {
Option.of(p.getString(HoodieWriteConfig.TBL_NAME))
} else {
Option.empty[String]()
}
})
val HIVE_TABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.table")
.defaultValue("unknown")
.withInferFunction(hiveTableOptKeyInferFunc)
.withDocumentation("table to sync to")
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: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.username")
.defaultValue("hive")
.withDocumentation("hive user name to use")
val HIVE_PASS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.password")
.defaultValue("hive")
.withDocumentation("hive password to use")
val HIVE_URL: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.jdbcurl")
.defaultValue("jdbc:hive2://localhost:10000")
.withDocumentation("Hive jdbc url")
val METASTORE_URIS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.metastore.uris")
.defaultValue("thrift://localhost:9083")
.withDocumentation("Hive metastore url")
val hivePartitionFieldsInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => {
if (p.contains(PARTITIONPATH_FIELD)) {
Option.of(p.getString(PARTITIONPATH_FIELD))
} else {
Option.empty[String]()
}
})
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.")
.withInferFunction(hivePartitionFieldsInferFunc)
val hivePartitionExtractorInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => {
if (!p.contains(PARTITIONPATH_FIELD)) {
Option.of(classOf[NonPartitionedExtractor].getName)
} else {
val numOfPartFields = p.getString(PARTITIONPATH_FIELD).split(",").length
if (numOfPartFields == 1 && p.contains(HIVE_STYLE_PARTITIONING) && p.getString(HIVE_STYLE_PARTITIONING) == "true") {
Option.of(classOf[HiveStylePartitionValueExtractor].getName)
} else {
Option.of(classOf[MultiPartKeysValueExtractor].getName)
}
}
})
val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.partition_extractor_class")
.defaultValue(classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName)
.withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, "
+ "default 'SlashEncodedDayPartitionValueExtractor'.")
.withInferFunction(hivePartitionExtractorInferFunc)
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: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.use_pre_apache_input_format")
.defaultValue("false")
.withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. "
+ "Use this when you are in the process of migrating from "
+ "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format")
/**
* @deprecated Hive Specific Configs are moved to {@link HiveSyncConfig}
*/
@Deprecated
val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_ENABLED
@Deprecated
val META_SYNC_ENABLED: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ENABLED
@Deprecated
val HIVE_DATABASE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_DATABASE_NAME
@Deprecated
val hiveTableOptKeyInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.TABLE_NAME_INFERENCE_FUNCTION
@Deprecated
val HIVE_TABLE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_TABLE_NAME
@Deprecated
val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT
@Deprecated
val HIVE_USER: ConfigProperty[String] = HiveSyncConfig.HIVE_USER
@Deprecated
val HIVE_PASS: ConfigProperty[String] = HiveSyncConfig.HIVE_PASS
@Deprecated
val HIVE_URL: ConfigProperty[String] = HiveSyncConfig.HIVE_URL
@Deprecated
val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfig.METASTORE_URIS
@Deprecated
val hivePartitionFieldsInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_FIELDS_INFERENCE_FUNCTION
@Deprecated
val HIVE_PARTITION_FIELDS: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS
@Deprecated
val hivePartitionExtractorInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_EXTRACTOR_CLASS_FUNCTION
@Deprecated
val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS
@Deprecated
val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION
@Deprecated
val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT
/** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */
@Deprecated
val HIVE_USE_JDBC: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.use_jdbc")
.defaultValue("true")
.deprecatedAfter("0.9.0")
.withDocumentation("Use JDBC when hive synchronization is enabled")
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: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.ignore_exceptions")
.defaultValue("false")
.withDocumentation("")
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: 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_TYPE: 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_USE_JDBC: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_JDBC
@Deprecated
val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE
@Deprecated
val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS
@Deprecated
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE
@Deprecated
val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE
/**
* Flag to indicate whether to use conditional syncing in HiveSync.
* If set true, the Hive sync procedure will only run if partition or schema changes are detected.
* By default true.
*/
val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.conditional_sync")
.defaultValue("false")
.withDocumentation("Enables conditional hive sync, where partition or schema change must exist to perform sync to hive.")
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.table_properties")
.noDefaultValue()
.withDocumentation("Additional properties to store with table.")
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.serde_properties")
.noDefaultValue()
.withDocumentation("Serde properties to hive table.")
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.sync_as_datasource")
.defaultValue("true")
.withDocumentation("")
@Deprecated
val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC
@Deprecated
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_PROPERTIES
@Deprecated
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES
@Deprecated
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE
// Create table as managed table
val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[Boolean] = ConfigProperty
.key("hoodie.datasource.hive_sync.create_managed_table")
.defaultValue(false)
.withDocumentation("Whether to sync the table as managed table.")
val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[Int] = ConfigProperty
.key("hoodie.datasource.hive_sync.batch_num")
.defaultValue(1000)
.withDocumentation("The number of partitions one batch when synchronous partitions to hive.")
val HIVE_SYNC_MODE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.mode")
.noDefaultValue()
.withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.")
val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[Boolean] = ConfigProperty
.key("hoodie.datasource.hive_sync.bucket_sync")
.defaultValue(false)
.withDocumentation("Whether sync hive metastore bucket specification when using bucket index." +
"The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'")
val HIVE_SYNC_COMMENT: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.sync_comment")
.defaultValue("false")
.withDocumentation("Whether to sync the table column comments while syncing the table.")
@Deprecated
val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE
@Deprecated
val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM
@Deprecated
val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_MODE
@Deprecated
val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC
@Deprecated
val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_COMMENT;
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty
@@ -584,19 +479,19 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */
@Deprecated
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HIVE_ASSUME_DATE_PARTITION.key()
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key()
/** @deprecated Use {@link HIVE_USE_PRE_APACHE_INPUT_FORMAT} and its methods instead */
@Deprecated
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HIVE_USE_PRE_APACHE_INPUT_FORMAT.key()
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key()
/** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */
@Deprecated
val HIVE_USE_JDBC_OPT_KEY = HIVE_USE_JDBC.key()
val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfig.HIVE_USE_JDBC.key()
/** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */
@Deprecated
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.key()
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key()
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
@Deprecated
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.key()
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key()
/** @deprecated Use {@link STREAMING_IGNORE_FAILED_BATCH} and its methods instead */
@Deprecated
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = STREAMING_IGNORE_FAILED_BATCH.key()
@@ -611,34 +506,34 @@ object DataSourceWriteOptions {
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue()
/** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
@Deprecated
val HIVE_SYNC_ENABLED_OPT_KEY = HIVE_SYNC_ENABLED.key()
val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfig.HIVE_SYNC_ENABLED.key()
/** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */
@Deprecated
val META_SYNC_ENABLED_OPT_KEY = META_SYNC_ENABLED.key()
val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()
/** @deprecated Use {@link HIVE_DATABASE} and its methods instead */
@Deprecated
val HIVE_DATABASE_OPT_KEY = HIVE_DATABASE.key()
val HIVE_DATABASE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()
/** @deprecated Use {@link HIVE_TABLE} and its methods instead */
@Deprecated
val HIVE_TABLE_OPT_KEY = HIVE_TABLE.key()
val HIVE_TABLE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()
/** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */
@Deprecated
val HIVE_BASE_FILE_FORMAT_OPT_KEY = HIVE_BASE_FILE_FORMAT.key()
val HIVE_BASE_FILE_FORMAT_OPT_KEY = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key()
/** @deprecated Use {@link HIVE_USER} and its methods instead */
@Deprecated
val HIVE_USER_OPT_KEY = HIVE_USER.key()
val HIVE_USER_OPT_KEY = HiveSyncConfig.HIVE_USER.key()
/** @deprecated Use {@link HIVE_PASS} and its methods instead */
@Deprecated
val HIVE_PASS_OPT_KEY = HIVE_PASS.key()
val HIVE_PASS_OPT_KEY = HiveSyncConfig.HIVE_PASS.key()
/** @deprecated Use {@link HIVE_URL} and its methods instead */
@Deprecated
val HIVE_URL_OPT_KEY = HIVE_URL.key()
val HIVE_URL_OPT_KEY = HiveSyncConfig.HIVE_URL.key()
/** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */
@Deprecated
val HIVE_PARTITION_FIELDS_OPT_KEY = HIVE_PARTITION_FIELDS.key()
val HIVE_PARTITION_FIELDS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key()
/** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */
@Deprecated
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HIVE_PARTITION_EXTRACTOR_CLASS.key()
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key()
/** @deprecated Use {@link KEYGENERATOR_CLASS_NAME} and its methods instead */
@Deprecated
@@ -748,60 +643,60 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HIVE_SYNC_ENABLED.defaultValue()
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfig.HIVE_SYNC_ENABLED.defaultValue()
/** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */
@Deprecated
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = META_SYNC_ENABLED.defaultValue()
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = HoodieSyncConfig.META_SYNC_ENABLED.defaultValue()
/** @deprecated Use {@link HIVE_DATABASE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_DATABASE_OPT_VAL = HIVE_DATABASE.defaultValue()
val DEFAULT_HIVE_DATABASE_OPT_VAL = HoodieSyncConfig.META_SYNC_DATABASE_NAME.defaultValue()
/** @deprecated Use {@link HIVE_TABLE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_TABLE_OPT_VAL = HIVE_TABLE.defaultValue()
val DEFAULT_HIVE_TABLE_OPT_VAL = HoodieSyncConfig.META_SYNC_TABLE_NAME.defaultValue()
/** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */
@Deprecated
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HIVE_BASE_FILE_FORMAT.defaultValue()
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue()
/** @deprecated Use {@link HIVE_USER} and its methods instead */
@Deprecated
val DEFAULT_HIVE_USER_OPT_VAL = HIVE_USER.defaultValue()
val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfig.HIVE_USER.defaultValue()
/** @deprecated Use {@link HIVE_PASS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_PASS_OPT_VAL = HIVE_PASS.defaultValue()
val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfig.HIVE_PASS.defaultValue()
/** @deprecated Use {@link HIVE_URL} and its methods instead */
@Deprecated
val DEFAULT_HIVE_URL_OPT_VAL = HIVE_URL.defaultValue()
val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfig.HIVE_URL.defaultValue()
/** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HIVE_PARTITION_FIELDS.defaultValue()
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.defaultValue()
/** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HIVE_PARTITION_EXTRACTOR_CLASS.defaultValue()
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.defaultValue()
/** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */
@Deprecated
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HIVE_ASSUME_DATE_PARTITION.defaultValue()
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.defaultValue()
@Deprecated
val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false"
/** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */
@Deprecated
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HIVE_USE_JDBC.defaultValue()
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfig.HIVE_USE_JDBC.defaultValue()
/** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.defaultValue()
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue()
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.defaultValue()
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue()
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
@Deprecated
val HIVE_SKIP_RO_SUFFIX = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
val HIVE_SKIP_RO_SUFFIX = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
@Deprecated
val HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.key()
val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key()
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
/** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */
@Deprecated
val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key()

View File

@@ -19,7 +19,6 @@ package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.avro.reflect.AvroSchema
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.conf.HiveConf
@@ -32,7 +31,7 @@ import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model._
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils}
import org.apache.hudi.common.util.{CommitUtils, StringUtils}
import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME}
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieException
@@ -40,24 +39,21 @@ import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRo
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.index.SparkHoodieIndexFactory
import org.apache.hudi.internal.DataSourceInternalWriterHelper
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.sync.common.AbstractSyncTool
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.hudi.sync.common.util.SyncUtilHelpers
import org.apache.hudi.table.BulkInsertPartitioner
import org.apache.log4j.LogManager
import org.apache.spark.SPARK_VERSION
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.hive.HiveExternalCatalog
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql._
import org.apache.spark.SparkContext
import org.apache.spark.sql.internal.StaticSQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.{SPARK_VERSION, SparkContext}
import java.util.Properties
import scala.collection.JavaConversions._
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
object HoodieSparkSqlWriter {
@@ -512,14 +508,7 @@ object HoodieSparkSqlWriter {
+ " To use row writer please switch to spark 2 or spark 3")
}
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params)
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
val metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
val syncHiveSuccess =
if (hiveSyncEnabled || metaSyncEnabled) {
metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
} else {
true
}
val syncHiveSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
(syncHiveSuccess, common.util.Option.ofNullable(instantTime))
}
@@ -558,57 +547,10 @@ object HoodieSparkSqlWriter {
}
}
private def syncHive(basePath: Path, fs: FileSystem, hoodieConfig: HoodieConfig, sqlConf: SQLConf): Boolean = {
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, hoodieConfig, sqlConf)
val hiveConf: HiveConf = new HiveConf()
hiveConf.addResource(fs.getConf)
if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) {
hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris)
}
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable()
true
}
private def buildSyncConfig(basePath: Path, hoodieConfig: HoodieConfig, sqlConf: SQLConf): HiveSyncConfig = {
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
hiveSyncConfig.basePath = basePath.toString
hiveSyncConfig.baseFileFormat = hoodieConfig.getString(HIVE_BASE_FILE_FORMAT)
hiveSyncConfig.usePreApacheInputFormat =
hoodieConfig.getStringOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT).toBoolean
hiveSyncConfig.databaseName = hoodieConfig.getString(HIVE_DATABASE)
hiveSyncConfig.tableName = hoodieConfig.getString(HIVE_TABLE)
hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER)
hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS)
hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL)
hiveSyncConfig.metastoreUris = hoodieConfig.getStringOrDefault(METASTORE_URIS)
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE,
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean
hiveSyncConfig.partitionFields =
ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS)
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC)
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean
hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING).toBoolean
hiveSyncConfig.batchSyncNum = hoodieConfig.getStringOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM).toInt
hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
hiveSyncConfig.sparkSchemaLengthThreshold = sqlConf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD)
hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)
hiveSyncConfig.syncMode = hoodieConfig.getString(HIVE_SYNC_MODE)
hiveSyncConfig.serdeProperties = hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES)
hiveSyncConfig.tableProperties = hoodieConfig.getString(HIVE_TABLE_PROPERTIES)
hiveSyncConfig.sparkVersion = SPARK_VERSION
hiveSyncConfig.syncComment = hoodieConfig.getStringOrDefault(HIVE_SYNC_COMMENT).toBoolean
hiveSyncConfig
}
private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path,
schema: StructType): Boolean = {
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfig.HIVE_SYNC_ENABLED).toBoolean
var metaSyncEnabled = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_ENABLED).toBoolean
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
@@ -617,29 +559,23 @@ object HoodieSparkSqlWriter {
metaSyncEnabled = true
syncClientToolClassSet += classOf[HiveSyncTool].getName
}
var metaSyncSuccess = true
if (metaSyncEnabled) {
val fs = basePath.getFileSystem(spark.sessionState.newHadoopConf())
val properties = new TypedProperties()
properties.putAll(hoodieConfig.getProps)
properties.put(HiveSyncConfig.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key, spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD).toString)
properties.put(HoodieSyncConfig.META_SYNC_SPARK_VERSION.key, SPARK_VERSION)
properties.put(HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA.key, hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE))
val hiveConf: HiveConf = new HiveConf()
hiveConf.addResource(fs.getConf)
syncClientToolClassSet.foreach(impl => {
val syncSuccess = impl.trim match {
case "org.apache.hudi.hive.HiveSyncTool" => {
log.info("Syncing to Hive Metastore (URL: " + hoodieConfig.getString(HIVE_URL) + ")")
syncHive(basePath, fs, hoodieConfig, spark.sessionState.conf)
true
}
case _ => {
val properties = new Properties()
properties.putAll(hoodieConfig.getProps)
properties.put("basePath", basePath.toString)
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
syncHoodie.syncHoodieTable()
true
}
}
metaSyncSuccess = metaSyncSuccess && syncSuccess
SyncUtilHelpers.runHoodieMetaSync(impl.trim, properties, hiveConf, fs, basePath.toString, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue)
})
}
metaSyncSuccess
true
}
/**

View File

@@ -18,13 +18,14 @@
package org.apache.hudi
import java.util.Properties
import org.apache.hudi.DataSourceOptionsHelper.allAlternatives
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties}
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.hudi.command.SqlKeyGenerator
@@ -64,21 +65,21 @@ object HoodieWriterUtils {
hoodieConfig.setDefaultValue(STREAMING_RETRY_INTERVAL_MS)
hoodieConfig.setDefaultValue(STREAMING_IGNORE_FAILED_BATCH)
hoodieConfig.setDefaultValue(META_SYNC_CLIENT_TOOL_CLASS_NAME)
hoodieConfig.setDefaultValue(HIVE_SYNC_ENABLED)
hoodieConfig.setDefaultValue(META_SYNC_ENABLED)
hoodieConfig.setDefaultValue(HIVE_DATABASE)
hoodieConfig.setDefaultValue(HIVE_TABLE)
hoodieConfig.setDefaultValue(HIVE_BASE_FILE_FORMAT)
hoodieConfig.setDefaultValue(HIVE_USER)
hoodieConfig.setDefaultValue(HIVE_PASS)
hoodieConfig.setDefaultValue(HIVE_URL)
hoodieConfig.setDefaultValue(METASTORE_URIS)
hoodieConfig.setDefaultValue(HIVE_PARTITION_FIELDS)
hoodieConfig.setDefaultValue(HIVE_PARTITION_EXTRACTOR_CLASS)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_ENABLED)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_ENABLED)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT)
hoodieConfig.setDefaultValue(HiveSyncConfig.METASTORE_URIS)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USER)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_PASS)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_URL)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS)
hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING)
hoodieConfig.setDefaultValue(HIVE_USE_JDBC)
hoodieConfig.setDefaultValue(HIVE_CREATE_MANAGED_TABLE)
hoodieConfig.setDefaultValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USE_JDBC)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE)
hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE)
hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE)
hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE)

View File

@@ -22,8 +22,9 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.util.PartitionPathEncodeUtils
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.MultiPartKeysValueExtractor
import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor}
import org.apache.hudi.hive.ddl.HiveSyncMode
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.Resolver
@@ -102,15 +103,15 @@ case class AlterHoodieTableDropPartitionCommand(
RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
PARTITIONPATH_FIELD.key -> partitionFields,
HIVE_SYNC_ENABLED.key -> enableHive.toString,
META_SYNC_ENABLED.key -> enableHive.toString,
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
HIVE_USE_JDBC.key -> "false",
HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table,
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HIVE_PARTITION_FIELDS.key -> partitionFields,
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
HiveSyncConfig.HIVE_USE_JDBC.key -> "false",
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hoodieCatalogTable.table.identifier.table,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName
)
}
}