1
0

[HUDI-3730] Improve meta sync class design and hierarchies (#5854)

* [HUDI-3730] Improve meta sync class design and hierarchies (#5754)
* Implements class design proposed in RFC-55

Co-authored-by: jian.feng <fengjian428@gmial.com>
Co-authored-by: jian.feng <jian.feng@shopee.com>
This commit is contained in:
Shiyan Xu
2022-07-03 04:17:25 -05:00
committed by GitHub
parent c00ea84985
commit c0e1587966
86 changed files with 2977 additions and 2877 deletions

View File

@@ -18,9 +18,6 @@
package org.apache.hudi;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.HoodieWriteResult;
import org.apache.hudi.client.SparkRDDWriteClient;
@@ -36,30 +33,27 @@ import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.TablePathUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodiePayloadConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
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;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.hive.HiveExternalCatalog;
import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -270,63 +264,6 @@ 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();
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.usePreApacheInputFormat =
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().key(), DataSourceWriteOptions.HIVE_USER().defaultValue());
hiveSyncConfig.hivePass =
props.getString(DataSourceWriteOptions.HIVE_PASS().key(), DataSourceWriteOptions.HIVE_PASS().defaultValue());
hiveSyncConfig.jdbcUrl =
props.getString(DataSourceWriteOptions.HIVE_URL().key(), DataSourceWriteOptions.HIVE_URL().defaultValue());
hiveSyncConfig.metastoreUris =
props.getString(DataSourceWriteOptions.METASTORE_URIS().key(), DataSourceWriteOptions.METASTORE_URIS().defaultValue());
hiveSyncConfig.partitionFields =
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), ",", new ArrayList<>());
hiveSyncConfig.partitionValueExtractorClass =
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
SlashEncodedDayPartitionValueExtractor.class.getName());
hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC().key(),
DataSourceWriteOptions.HIVE_USE_JDBC().defaultValue()));
if (props.containsKey(DataSourceWriteOptions.HIVE_SYNC_MODE().key())) {
hiveSyncConfig.syncMode = props.getString(DataSourceWriteOptions.HIVE_SYNC_MODE().key());
}
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_FOR_READ_OPTIMIZED_TABLE().key(),
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().defaultValue()));
hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().key(),
DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().defaultValue()));
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(),
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())) {
hiveSyncConfig.sparkVersion = props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION());
}
hiveSyncConfig.syncComment = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SYNC_COMMENT().key(),
DataSourceWriteOptions.HIVE_SYNC_COMMENT().defaultValue()));
return hiveSyncConfig;
}
/**
* Checks whether default value (false) of "hoodie.parquet.writelegacyformat.enabled" should be
* overridden in case:

View File

@@ -26,7 +26,7 @@ import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.common.util.Option
import org.apache.hudi.common.util.ValidationUtils.checkState
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, HiveSyncTool}
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.sync.common.HoodieSyncConfig
@@ -413,7 +413,7 @@ object DataSourceWriteOptions {
* @deprecated Hive Specific Configs are moved to {@link HiveSyncConfig}
*/
@Deprecated
val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_ENABLED
val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_ENABLED
@Deprecated
val META_SYNC_ENABLED: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ENABLED
@Deprecated
@@ -425,13 +425,13 @@ object DataSourceWriteOptions {
@Deprecated
val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT
@Deprecated
val HIVE_USER: ConfigProperty[String] = HiveSyncConfig.HIVE_USER
val HIVE_USER: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USER
@Deprecated
val HIVE_PASS: ConfigProperty[String] = HiveSyncConfig.HIVE_PASS
val HIVE_PASS: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_PASS
@Deprecated
val HIVE_URL: ConfigProperty[String] = HiveSyncConfig.HIVE_URL
val HIVE_URL: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_URL
@Deprecated
val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfig.METASTORE_URIS
val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfigHolder.METASTORE_URIS
@Deprecated
val hivePartitionFieldsInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_FIELDS_INFERENCE_FUNCTION
@Deprecated
@@ -443,19 +443,19 @@ object DataSourceWriteOptions {
@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
val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfigHolder.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] = HiveSyncConfig.HIVE_USE_JDBC
val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_JDBC
@Deprecated
val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE
val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE
@Deprecated
val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS
val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS
@Deprecated
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE
@Deprecated
val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE
val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE
/**
* Flag to indicate whether to use conditional syncing in HiveSync.
@@ -465,23 +465,23 @@ object DataSourceWriteOptions {
@Deprecated
val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC
@Deprecated
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_PROPERTIES
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES
@Deprecated
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES
@Deprecated
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE
// Create table as managed table
@Deprecated
val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE
val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE
@Deprecated
val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM
val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM
@Deprecated
val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_MODE
val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_MODE
@Deprecated
val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC
val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC
@Deprecated
val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_COMMENT;
val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_COMMENT;
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty
@@ -506,16 +506,16 @@ object DataSourceWriteOptions {
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 = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key()
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key()
/** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */
@Deprecated
val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfig.HIVE_USE_JDBC.key()
val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_JDBC.key()
/** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */
@Deprecated
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key()
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.key()
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
@Deprecated
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key()
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfigHolder.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()
@@ -530,7 +530,7 @@ 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 = HiveSyncConfig.HIVE_SYNC_ENABLED.key()
val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key()
/** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */
@Deprecated
val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_ENABLED.key()
@@ -545,13 +545,13 @@ object DataSourceWriteOptions {
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 = HiveSyncConfig.HIVE_USER.key()
val HIVE_USER_OPT_KEY = HiveSyncConfigHolder.HIVE_USER.key()
/** @deprecated Use {@link HIVE_PASS} and its methods instead */
@Deprecated
val HIVE_PASS_OPT_KEY = HiveSyncConfig.HIVE_PASS.key()
val HIVE_PASS_OPT_KEY = HiveSyncConfigHolder.HIVE_PASS.key()
/** @deprecated Use {@link HIVE_URL} and its methods instead */
@Deprecated
val HIVE_URL_OPT_KEY = HiveSyncConfig.HIVE_URL.key()
val HIVE_URL_OPT_KEY = HiveSyncConfigHolder.HIVE_URL.key()
/** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */
@Deprecated
val HIVE_PARTITION_FIELDS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key()
@@ -667,7 +667,7 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfig.HIVE_SYNC_ENABLED.defaultValue()
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfigHolder.HIVE_SYNC_ENABLED.defaultValue()
/** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */
@Deprecated
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = HoodieSyncConfig.META_SYNC_ENABLED.defaultValue()
@@ -682,13 +682,13 @@ object DataSourceWriteOptions {
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 = HiveSyncConfig.HIVE_USER.defaultValue()
val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfigHolder.HIVE_USER.defaultValue()
/** @deprecated Use {@link HIVE_PASS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfig.HIVE_PASS.defaultValue()
val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfigHolder.HIVE_PASS.defaultValue()
/** @deprecated Use {@link HIVE_URL} and its methods instead */
@Deprecated
val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfig.HIVE_URL.defaultValue()
val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfigHolder.HIVE_URL.defaultValue()
/** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.defaultValue()
@@ -702,25 +702,25 @@ object DataSourceWriteOptions {
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 = HiveSyncConfig.HIVE_USE_JDBC.defaultValue()
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfigHolder.HIVE_USE_JDBC.defaultValue()
/** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue()
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.defaultValue()
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue()
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfigHolder.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 = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
val HIVE_SKIP_RO_SUFFIX = HiveSyncConfigHolder.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 = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfigHolder.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 = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key()
val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key()
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfigHolder.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

@@ -36,7 +36,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool}
import org.apache.hudi.index.SparkHoodieIndexFactory
import org.apache.hudi.internal.DataSourceInternalWriterHelper
import org.apache.hudi.internal.schema.InternalSchema
@@ -600,7 +600,7 @@ object HoodieSparkSqlWriter {
private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path,
schema: StructType): Boolean = {
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfig.HIVE_SYNC_ENABLED).toBoolean
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfigHolder.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)
@@ -616,7 +616,7 @@ object HoodieSparkSqlWriter {
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT);
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(HiveSyncConfigHolder.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))

View File

@@ -17,20 +17,19 @@
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, HoodieCommonConfig, HoodieConfig, TypedProperties}
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieCommonConfig, HoodieConfig}
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.hive.HiveSyncConfigHolder
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.hudi.command.SqlKeyGenerator
import java.util.Properties
import scala.collection.JavaConversions.mapAsJavaMap
import scala.collection.JavaConverters._
@@ -67,21 +66,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(HiveSyncConfig.HIVE_SYNC_ENABLED)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.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(HiveSyncConfigHolder.METASTORE_URIS)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USER)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_PASS)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_URL)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS)
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS)
hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USE_JDBC)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE)
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USE_JDBC)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE)
hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE)
hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE)
hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE)
hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE)

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
import org.apache.hudi.hive.ddl.HiveSyncMode
import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, MultiPartKeysValueExtractor}
import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.sql.InsertMode
import org.apache.hudi.sync.common.HoodieSyncConfig
@@ -38,9 +38,7 @@ import org.apache.spark.sql.hudi.command.{SqlKeyGenerator, ValidateDuplicateKeyP
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import java.util
import java.util.Locale
import scala.collection.JavaConverters._
trait ProvidesHoodieConfig extends Logging {
@@ -76,13 +74,13 @@ trait ProvidesHoodieConfig extends Logging {
OPERATION.key -> UPSERT_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp,
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode,
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName,
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName,
HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()),
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME),
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME),
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS),
HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString,
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"),
SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL
)
@@ -194,12 +192,12 @@ trait ProvidesHoodieConfig extends Logging {
HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn),
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFieldsStr,
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode,
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName,
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass,
HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()),
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME),
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME),
HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS),
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "200"),
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"),
SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL
@@ -231,13 +229,13 @@ trait ProvidesHoodieConfig extends Logging {
PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
PARTITIONPATH_FIELD.key -> partitionFields,
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode,
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName,
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString,
HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()),
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME),
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME),
HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString,
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS)
)
.filter { case (_, v) => v != null }
}
@@ -273,9 +271,9 @@ trait ProvidesHoodieConfig extends Logging {
OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp,
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString,
HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()),
HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString,
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"),
SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL
)
@@ -289,33 +287,20 @@ trait ProvidesHoodieConfig extends Logging {
}
def buildHiveSyncConfig(props: TypedProperties, hoodieCatalogTable: HoodieCatalogTable): HiveSyncConfig = {
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig
hiveSyncConfig.basePath = hoodieCatalogTable.tableLocation
hiveSyncConfig.baseFileFormat = hoodieCatalogTable.baseFileFormat
hiveSyncConfig.usePreApacheInputFormat = props.getBoolean(HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key, HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.defaultValue.toBoolean)
hiveSyncConfig.databaseName = hoodieCatalogTable.table.identifier.database.getOrElse("default")
if (props.containsKey(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)) {
hiveSyncConfig.tableName = props.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)
} else {
hiveSyncConfig.tableName = hoodieCatalogTable.table.identifier.table
}
hiveSyncConfig.syncMode = props.getString(HiveSyncConfig.HIVE_SYNC_MODE.key, HiveSyncMode.HMS.name())
hiveSyncConfig.hiveUser = props.getString(HiveSyncConfig.HIVE_USER.key, HiveSyncConfig.HIVE_USER.defaultValue)
hiveSyncConfig.hivePass = props.getString(HiveSyncConfig.HIVE_PASS.key, HiveSyncConfig.HIVE_PASS.defaultValue)
hiveSyncConfig.jdbcUrl = props.getString(HiveSyncConfig.HIVE_URL.key, HiveSyncConfig.HIVE_URL.defaultValue)
hiveSyncConfig.metastoreUris = props.getString(HiveSyncConfig.METASTORE_URIS.key, HiveSyncConfig.METASTORE_URIS.defaultValue)
hiveSyncConfig.partitionFields = props.getStringList(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key, ",", new util.ArrayList[String])
hiveSyncConfig.partitionValueExtractorClass = props.getString(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key, classOf[MultiPartKeysValueExtractor].getName)
if (props.containsKey(HiveSyncConfig.HIVE_SYNC_MODE.key)) hiveSyncConfig.syncMode = props.getString(HiveSyncConfig.HIVE_SYNC_MODE.key)
hiveSyncConfig.autoCreateDatabase = props.getString(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key, HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue).toBoolean
hiveSyncConfig.ignoreExceptions = props.getString(HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key, HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue).toBoolean
hiveSyncConfig.skipROSuffix = props.getString(HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key, HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean
hiveSyncConfig.supportTimestamp = props.getString(HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key, "true").toBoolean
hiveSyncConfig.isConditionalSync = props.getString(HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC.key, HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC.defaultValue).toBoolean
hiveSyncConfig.bucketSpec = if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key, HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue)) HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key), props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key))
else null
if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION)) hiveSyncConfig.sparkVersion = props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION)
hiveSyncConfig.syncComment = props.getString(DataSourceWriteOptions.HIVE_SYNC_COMMENT.key, DataSourceWriteOptions.HIVE_SYNC_COMMENT.defaultValue).toBoolean
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig(props)
hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_BASE_PATH, hoodieCatalogTable.tableLocation)
hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT, hoodieCatalogTable.baseFileFormat)
hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME, hoodieCatalogTable.table.identifier.database.getOrElse("default"))
hiveSyncConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME, hoodieCatalogTable.table.identifier.table)
hiveSyncConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS, classOf[MultiPartKeysValueExtractor].getName)
hiveSyncConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE, "true")
if (hiveSyncConfig.useBucketSync())
hiveSyncConfig.setValue(HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC,
HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key),
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key)))
if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION))
hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_SPARK_VERSION,
props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION))
hiveSyncConfig
}
}

View File

@@ -18,9 +18,6 @@
package org.apache.hudi.cli;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
@@ -38,7 +35,10 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HiveSyncTool;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.sync.common.HoodieSyncConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
@@ -48,6 +48,12 @@ import java.io.Serializable;
import java.util.HashMap;
import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD;
import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH;
/**
* Performs bootstrap from a non-hudi source.
@@ -115,7 +121,7 @@ public class BootstrapExecutorUtils implements Serializable {
// Add more defaults if full bootstrap requested
this.props.putIfAbsent(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key(),
DataSourceWriteOptions.PAYLOAD_CLASS_NAME().defaultValue());
/**
/*
* Schema provider that supplies the command for reading the input and writing out the target table.
*/
SchemaProvider schemaProvider = createSchemaProvider(cfg.schemaProviderClass, props, jssc);
@@ -165,14 +171,14 @@ public class BootstrapExecutorUtils implements Serializable {
if (cfg.enableHiveSync) {
TypedProperties metaProps = new TypedProperties();
metaProps.putAll(props);
metaProps.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), cfg.basePath);
metaProps.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);
if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key(), HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue())) {
metaProps.put(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()),
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())));
metaProps.put(META_SYNC_BASE_PATH.key(), cfg.basePath);
metaProps.put(META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);
if (props.getBoolean(HIVE_SYNC_BUCKET_SYNC.key(), HIVE_SYNC_BUCKET_SYNC.defaultValue())) {
metaProps.put(HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(BUCKET_INDEX_HASH_FIELD.key()),
props.getInteger(BUCKET_INDEX_NUM_BUCKETS.key())));
}
new HiveSyncTool(metaProps, configuration, fs).syncHoodieTable();
new HiveSyncTool(metaProps, configuration).syncHoodieTable();
}
}

View File

@@ -20,11 +20,11 @@ package org.apache.spark.sql.hudi.command
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.hive.HiveSyncConfigHolder
import org.apache.hudi.sql.InsertMode
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
import org.apache.hudi.sync.common.util.ConfigUtils
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
@@ -94,9 +94,9 @@ case class CreateHoodieTableAsSelectCommand(
val tblProperties = hoodieCatalogTable.catalogProperties
val options = Map(
HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString,
HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
HiveSyncConfig.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(newTable.properties.asJava),
HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString,
HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(newTable.properties.asJava),
DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(),
DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true"
)

View File

@@ -22,7 +22,7 @@ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.util.StringUtils
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.hive.HiveSyncConfigHolder
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, SparkAdapterSupport}
import org.apache.spark.sql._
@@ -467,13 +467,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName,
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode,
HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getString(HiveSyncConfigHolder.HIVE_SYNC_MODE),
HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> targetTableDb,
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> targetTableName,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString,
HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString,
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getString(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS),
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "200"), // set the default parallelism to 200 for sql
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"),
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"),

View File

@@ -50,6 +50,12 @@ import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_ENABLED;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
/**
* Sample program that writes & reads hoodie tables via the Spark datasource.
@@ -256,12 +262,12 @@ public class HoodieJavaApp {
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
writer = writer.option(META_SYNC_TABLE_NAME.key(), hiveTable)
.option(META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HIVE_URL.key(), hiveJdbcUrl)
.option(HIVE_USER.key(), hiveUser)
.option(HIVE_PASS.key(), hivePass)
.option(HIVE_SYNC_ENABLED.key(), "true");
if (nonPartitionedTable) {
writer = writer
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),

View File

@@ -46,6 +46,12 @@ import java.util.ArrayList;
import java.util.List;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_ENABLED;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
public class HoodieJavaGenerateApp {
@Parameter(names = {"--table-path", "-p"}, description = "Path for Hoodie sample table")
@@ -126,12 +132,12 @@ public class HoodieJavaGenerateApp {
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
writer = writer.option(META_SYNC_TABLE_NAME.key(), hiveTable)
.option(META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HIVE_URL.key(), hiveJdbcUrl)
.option(HIVE_USER.key(), hiveUser)
.option(HIVE_PASS.key(), hivePass)
.option(HIVE_SYNC_ENABLED.key(), "true");
if (nonPartitionedTable) {
writer = writer
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),

View File

@@ -54,6 +54,12 @@ import java.util.concurrent.Future;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_ENABLED;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
/**
* Sample program that writes & reads hoodie tables via the Spark datasource streaming.
@@ -383,12 +389,12 @@ public class HoodieJavaStreamingApp {
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
writer = writer.option(META_SYNC_TABLE_NAME.key(), hiveTable)
.option(META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HIVE_URL.key(), hiveJdbcUrl)
.option(HIVE_USER.key(), hiveUser)
.option(HIVE_PASS.key(), hivePass)
.option(HIVE_SYNC_ENABLED.key(), "true");
if (useMultiPartitionKeys) {
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),

View File

@@ -18,12 +18,6 @@
package org.apache.hudi;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.TypedProperties;
@@ -37,8 +31,14 @@ import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
@@ -54,7 +54,6 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.ArgumentCaptor;
import org.mockito.Captor;
import org.mockito.Mock;
@@ -70,18 +69,13 @@ import java.util.Map;
import java.util.stream.Stream;
import static org.apache.hudi.DataSourceUtils.tryOverrideParquetWriteLegacyFormatProperty;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
import static org.apache.hudi.hive.ddl.HiveSyncMode.HMS;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.times;
@@ -252,29 +246,6 @@ public class TestDataSourceUtils {
});
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testBuildHiveSyncConfig(boolean useSyncMode) {
TypedProperties props = new TypedProperties();
if (useSyncMode) {
props.setProperty(DataSourceWriteOptions.HIVE_SYNC_MODE().key(), HMS.name());
props.setProperty(DataSourceWriteOptions.HIVE_USE_JDBC().key(), String.valueOf(false));
}
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), HIVE_DATABASE);
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), HIVE_TABLE);
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, config.getBasePath(), PARQUET.name());
if (useSyncMode) {
assertFalse(hiveSyncConfig.useJdbc);
assertEquals(HMS.name(), hiveSyncConfig.syncMode);
} else {
assertTrue(hiveSyncConfig.useJdbc);
assertNull(hiveSyncConfig.syncMode);
}
assertEquals(HIVE_DATABASE, hiveSyncConfig.databaseName);
assertEquals(HIVE_TABLE, hiveSyncConfig.tableName);
}
private void setAndVerifyHoodieWriteClientWith(final String partitionerClassName) {
config = HoodieWriteConfig.newBuilder().withPath(config.getBasePath())
.withUserDefinedBulkInsertPartitionerClass(partitionerClassName)