1
0

[HUDI-2362] Add external config file support (#3416)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-11-18 01:59:26 -08:00
committed by GitHub
parent 8772cec4bd
commit 24def0b30d
25 changed files with 426 additions and 102 deletions

View File

@@ -22,7 +22,7 @@ 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.{HoodieConfig, TypedProperties}
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.spark.sql.SparkSession
@@ -47,6 +47,7 @@ object HoodieWriterUtils {
* @return
*/
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala
Map(OPERATION.key -> OPERATION.defaultValue,
TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
@@ -81,7 +82,7 @@ object HoodieWriterUtils {
ENABLE_ROW_WRITER.key -> ENABLE_ROW_WRITER.defaultValue,
RECONCILE_SCHEMA.key -> RECONCILE_SCHEMA.defaultValue.toString,
DROP_PARTITION_COLUMNS.key -> DROP_PARTITION_COLUMNS.defaultValue
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
) ++ globalProps ++ DataSourceOptionsHelper.translateConfigurations(parameters)
}
def toProperties(params: Map[String, String]): TypedProperties = {
@@ -170,4 +171,23 @@ object HoodieWriterUtils {
}
}
}
val sparkDatasourceConfigsToTableConfigsMap = Map(
TABLE_NAME -> HoodieTableConfig.NAME,
TABLE_TYPE -> HoodieTableConfig.TYPE,
PRECOMBINE_FIELD -> HoodieTableConfig.PRECOMBINE_FIELD,
PARTITIONPATH_FIELD -> HoodieTableConfig.PARTITION_FIELDS,
RECORDKEY_FIELD -> HoodieTableConfig.RECORDKEY_FIELDS,
PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME
)
def mappingSparkDatasourceConfigsToTableConfigs(options: Map[String, String]): Map[String, String] = {
val includingTableConfigs = scala.collection.mutable.Map() ++ options
sparkDatasourceConfigsToTableConfigsMap.foreach(kv => {
if (options.containsKey(kv._1.key)) {
includingTableConfigs(kv._2.key) = options(kv._1.key)
includingTableConfigs.remove(kv._1.key)
}
})
includingTableConfigs.toMap
}
}

View File

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.SparkAdapterSupport
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.config.DFSPropertiesConfiguration
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.HoodieRecord
@@ -273,7 +274,7 @@ object HoodieSqlUtils extends SparkAdapterSupport {
*/
def withSparkConf(spark: SparkSession, options: Map[String, String])
(baseConfig: Map[String, String] = Map.empty): Map[String, String] = {
baseConfig ++ // Table options has the highest priority
baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority
(spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options))
.filterKeys(_.startsWith("hoodie."))
}

View File

@@ -43,7 +43,7 @@ class AlterHoodieTableRenameCommand(
.setConf(hadoopConf).build()
// Init table with new name.
HoodieTableMetaClient.withPropertyBuilder()
.fromProperties(metaClient.getTableConfig.getProps)
.fromProperties(metaClient.getTableConfig.getProps(true))
.setTableName(newName.table)
.initTable(hadoopConf, path)
// Call AlterTableRenameCommand#run to rename table in meta.

View File

@@ -19,9 +19,9 @@ package org.apache.spark.sql.hudi.command
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
import org.apache.hudi.{DataSourceWriteOptions, HoodieWriterUtils, SparkAdapterSupport}
import org.apache.hudi.HoodieWriterUtils._
import org.apache.hudi.common.config.DFSPropertiesConfiguration
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.hadoop.HoodieParquetInputFormat
@@ -47,7 +47,7 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.{SPARK_VERSION, SparkConf}
import java.util.{Locale, Properties}
import scala.collection.JavaConversions.mapAsJavaMap
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.control.NonFatal
@@ -100,7 +100,10 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
// if CTAS, we treat the table we just created as nonexistent
val isTableExists = if (ctas) false else tableExistsInPath(path, conf)
var existingTableConfig = Map.empty[String, String]
val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(tblProperties)
val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala.toMap
val globalSqlProps = HoodieOptionConfig.mappingTableConfigToSqlOption(
HoodieWriterUtils.mappingSparkDatasourceConfigsToTableConfigs(globalProps))
val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(globalSqlProps ++ tblProperties)
val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(tblProperties)
// get final schema and parameters
@@ -341,7 +344,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
}
} else {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true"
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue()
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue
}
if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) {
@@ -374,7 +377,7 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
// Save all the table config to the hoodie.properties.
val parameters = originTableConfig ++ tableOptions
val parameters = HoodieWriterUtils.mappingSparkDatasourceConfigsToTableConfigs(originTableConfig ++ tableOptions)
val properties = new Properties()
properties.putAll(parameters.asJava)
HoodieTableMetaClient.withPropertyBuilder()

View File

@@ -17,7 +17,7 @@
package org.apache.spark.sql.hudi.command
import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _}
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME

View File

@@ -45,7 +45,7 @@ class TruncateHoodieTableCommand(
// Create MetaClient
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
.setConf(hadoopConf).build()
Some(metaClient.getTableConfig.getProps)
Some(metaClient.getTableConfig.getProps(true))
} else {
None
}

View File

@@ -0,0 +1,23 @@
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
# Default system properties included when running Hudi jobs.
# This is useful for setting default environmental settings.
# Example:
hoodie.datasource.write.table.type MERGE_ON_READ
hoodie.datasource.write.hive_style_partitioning false

View File

@@ -0,0 +1,104 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.config.DFSPropertiesConfiguration
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import java.io.File
import java.nio.file.{Files, Paths}
import org.scalatest.BeforeAndAfter
class TestSqlConf extends TestHoodieSqlBase with BeforeAndAfter {
def setEnv(key: String, value: String): String = {
val field = System.getenv().getClass.getDeclaredField("m")
field.setAccessible(true)
val map = field.get(System.getenv()).asInstanceOf[java.util.Map[java.lang.String, java.lang.String]]
map.put(key, value)
}
test("Test Hudi Conf") {
withTempDir { tmp =>
val tableName = generateTableName
val tablePath = tmp.getCanonicalPath
val partitionVal = "2021"
// Create table
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long,
| year string
|) using hudi
| partitioned by (year)
| location '$tablePath'
| options (
| primaryKey ='id',
| preCombineField = 'ts'
| )
""".stripMargin)
// First merge with a extra input field 'flag' (insert a new record)
spark.sql(
s"""
| merge into $tableName
| using (
| select 1 as id, 'a1' as name, 10 as price, 1000 as ts, '1' as flag, $partitionVal as year
| ) s0
| on s0.id = $tableName.id
| when matched and flag = '1' then update set
| id = s0.id, name = s0.name, price = s0.price, ts = s0.ts, year = s0.year
| when not matched and flag = '1' then insert *
""".stripMargin)
checkAnswer(s"select id, name, price, ts, year from $tableName")(
Seq(1, "a1", 10.0, 1000, partitionVal)
)
// By default, Spark DML would set table type to COW and use Hive style partitioning, here we
// set table type to MOR and disable Hive style partitioning in the hudi conf file, and check
// if Hudi DML can load these configs correctly
assertResult(true)(Files.exists(Paths.get(s"$tablePath/$partitionVal")))
assertResult(HoodieTableType.MERGE_ON_READ)(new HoodieTableConfig(
new Path(tablePath).getFileSystem(new Configuration),
s"$tablePath/" + HoodieTableMetaClient.METAFOLDER_NAME,
HoodieTableConfig.PAYLOAD_CLASS_NAME.defaultValue).getTableType)
// delete the record
spark.sql(s"delete from $tableName where year = $partitionVal")
val cnt = spark.sql(s"select * from $tableName where year = $partitionVal").count()
assertResult(0)(cnt)
}
}
before {
val testPropsFilePath = new File("src/test/resources/external-config").getAbsolutePath
setEnv(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME, testPropsFilePath)
DFSPropertiesConfiguration.refreshGlobalProps()
}
after {
DFSPropertiesConfiguration.clearGlobalProps()
}
}