[HUDI-4001] Filter the properties should not be used when create table for Spark SQL (#5495)
This commit is contained in:
@@ -18,6 +18,7 @@
|
||||
package org.apache.spark.sql.catalyst.catalog
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils
|
||||
import org.apache.hudi.DataSourceWriteOptions.OPERATION
|
||||
import org.apache.hudi.HoodieWriterUtils._
|
||||
import org.apache.hudi.common.config.DFSPropertiesConfiguration
|
||||
import org.apache.hudi.common.model.HoodieTableType
|
||||
@@ -321,6 +322,8 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten
|
||||
}
|
||||
|
||||
object HoodieCatalogTable {
|
||||
// The properties should not be used when create table
|
||||
val needFilterProps: List[String] = List(HoodieTableConfig.DATABASE_NAME.key, HoodieTableConfig.NAME.key, OPERATION.key)
|
||||
|
||||
def apply(sparkSession: SparkSession, tableIdentifier: TableIdentifier): HoodieCatalogTable = {
|
||||
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
|
||||
|
||||
@@ -255,8 +255,7 @@ trait ProvidesHoodieConfig extends Logging {
|
||||
val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf)
|
||||
val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable)
|
||||
|
||||
// operation can not be overwrite
|
||||
val options = hoodieCatalogTable.catalogProperties.-(OPERATION.key())
|
||||
val options = hoodieCatalogTable.catalogProperties
|
||||
|
||||
withSparkConf(sparkSession, options) {
|
||||
Map(
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
|
||||
import org.apache.spark.sql.catalyst.catalog._
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
|
||||
import org.apache.spark.sql.hive.HiveClientUtils
|
||||
import org.apache.spark.sql.hive.HiveExternalCatalog._
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive
|
||||
@@ -130,8 +131,9 @@ object CreateHoodieTableCommand {
|
||||
.copy(table = tableName, database = Some(newDatabaseName))
|
||||
|
||||
val partitionColumnNames = hoodieCatalogTable.partitionSchema.map(_.name)
|
||||
// append pk, preCombineKey, type to the properties of table
|
||||
val newTblProperties = hoodieCatalogTable.catalogProperties ++ HoodieOptionConfig.extractSqlOptions(properties)
|
||||
// Remove some properties should not be used;append pk, preCombineKey, type to the properties of table
|
||||
val newTblProperties =
|
||||
hoodieCatalogTable.catalogProperties.--(needFilterProps) ++ HoodieOptionConfig.extractSqlOptions(properties)
|
||||
val newTable = table.copy(
|
||||
identifier = newTableIdentifier,
|
||||
storage = newStorage,
|
||||
|
||||
Reference in New Issue
Block a user