1
0

[HUDI-2057] CTAS Generate An External Table When Create Managed Table (#3146)

This commit is contained in:
pengzhiwei
2021-07-03 15:55:36 +08:00
committed by GitHub
parent 7173d1338a
commit 4f215e2938
8 changed files with 63 additions and 3 deletions

View File

@@ -439,8 +439,8 @@ object HoodieSparkSqlWriter {
serdeProp.put(ConfigUtils.SPARK_QUERY_AS_RT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
hiveSyncConfig.serdeProperties = ConfigUtils.configToString(serdeProp)
}
hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)
hiveSyncConfig
}

View File

@@ -73,6 +73,8 @@ object HoodieWriterUtils {
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.defaultValue,
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue,
HIVE_USE_JDBC_OPT_KEY.key -> HIVE_USE_JDBC_OPT_KEY.defaultValue,
HIVE_CREATE_MANAGED_TABLE.key() -> HIVE_CREATE_MANAGED_TABLE.defaultValue.toString,
HIVE_SYNC_AS_DATA_SOURCE_TABLE.key() -> HIVE_SYNC_AS_DATA_SOURCE_TABLE.defaultValue(),
ASYNC_COMPACT_ENABLE_OPT_KEY.key -> ASYNC_COMPACT_ENABLE_OPT_KEY.defaultValue,
ENABLE_ROW_WRITER_OPT_KEY.key -> ENABLE_ROW_WRITER_OPT_KEY.defaultValue
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)

View File

@@ -19,6 +19,7 @@ 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.spark.sql.{Row, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
@@ -69,6 +70,9 @@ case class CreateHoodieTableAsSelectCommand(
// Execute the insert query
try {
// Set if sync as a managed table.
sparkSession.sessionState.conf.setConfString(DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key(),
(table.tableType == CatalogTableType.MANAGED).toString)
val success = InsertIntoHoodieTableCommand.run(sparkSession, tableWithSchema, reOrderedQuery, Map.empty,
mode == SaveMode.Overwrite, refreshTable = false)
if (success) {