diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index b35e24042..eca73be0b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -76,6 +76,7 @@ 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, @@ -193,6 +194,7 @@ 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, @@ -257,6 +259,7 @@ trait ProvidesHoodieConfig extends Logging { val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) val options = hoodieCatalogTable.catalogProperties + val enableHive = isEnableHive(sparkSession) withSparkConf(sparkSession, options) { Map( @@ -269,6 +272,8 @@ trait ProvidesHoodieConfig extends Logging { SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, 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, HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"), diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 1376445bd..f7c62adc6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -467,6 +467,7 @@ 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, HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> targetTableDb, HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> targetTableName,