[HUDI-1415] Read Hoodie Table As Spark DataSource Table (#2283)
This commit is contained in:
@@ -36,6 +36,7 @@ import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP, DEFAULT_BOOTSTRAP_INDEX_CLASS}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hive.util.ConfigUtils
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||
@@ -44,7 +45,10 @@ import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.SparkContext
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
@@ -220,7 +224,8 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
|
||||
// Check for errors and commit the write.
|
||||
val (writeSuccessful, compactionInstant) =
|
||||
commitAndPerformPostOperations(writeResult, parameters, writeClient, tableConfig, jsc,
|
||||
commitAndPerformPostOperations(sqlContext.sparkSession, df.schema,
|
||||
writeResult, parameters, writeClient, tableConfig, jsc,
|
||||
TableInstantInfo(basePath, instantTime, commitActionType, operation))
|
||||
|
||||
def unpersistRdd(rdd: RDD[_]): Unit = {
|
||||
@@ -305,7 +310,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
} finally {
|
||||
writeClient.close()
|
||||
}
|
||||
val metaSyncSuccess = metaSync(parameters, basePath, jsc.hadoopConfiguration)
|
||||
val metaSyncSuccess = metaSync(sqlContext.sparkSession, parameters, basePath, df.schema)
|
||||
metaSyncSuccess
|
||||
}
|
||||
|
||||
@@ -346,12 +351,13 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
}
|
||||
val hiveSyncEnabled = params.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val metaSyncEnabled = params.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val syncHiveSucess = if (hiveSyncEnabled || metaSyncEnabled) {
|
||||
metaSync(params, basePath, sqlContext.sparkContext.hadoopConfiguration)
|
||||
val syncHiveSuccess =
|
||||
if (hiveSyncEnabled || metaSyncEnabled) {
|
||||
metaSync(sqlContext.sparkSession, parameters, basePath, df.schema)
|
||||
} else {
|
||||
true
|
||||
}
|
||||
(syncHiveSucess, common.util.Option.ofNullable(instantTime))
|
||||
(syncHiveSuccess, common.util.Option.ofNullable(instantTime))
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
@@ -398,7 +404,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = {
|
||||
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
|
||||
hiveSyncConfig.basePath = basePath.toString
|
||||
hiveSyncConfig.baseFileFormat = parameters(HIVE_BASE_FILE_FORMAT_OPT_KEY);
|
||||
hiveSyncConfig.baseFileFormat = parameters(HIVE_BASE_FILE_FORMAT_OPT_KEY)
|
||||
hiveSyncConfig.usePreApacheInputFormat =
|
||||
parameters.get(HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY).exists(r => r.toBoolean)
|
||||
hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY)
|
||||
@@ -417,17 +423,77 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
hiveSyncConfig.autoCreateDatabase = parameters.get(HIVE_AUTO_CREATE_DATABASE_OPT_KEY).exists(r => r.toBoolean)
|
||||
hiveSyncConfig.decodePartition = parameters.getOrElse(URL_ENCODE_PARTITIONING_OPT_KEY,
|
||||
DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL).toBoolean
|
||||
|
||||
val syncAsDtaSourceTable = parameters.getOrElse(DataSourceWriteOptions.HIVE_SYNC_AS_DATA_SOURCE_TABLE,
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
if (syncAsDtaSourceTable) {
|
||||
hiveSyncConfig.tableProperties = parameters.getOrElse(HIVE_TABLE_PROPERTIES, null)
|
||||
hiveSyncConfig.serdeProperties = createSqlTableSerdeProperties(parameters, basePath.toString)
|
||||
}
|
||||
hiveSyncConfig
|
||||
}
|
||||
|
||||
private def metaSync(parameters: Map[String, String],
|
||||
basePath: Path,
|
||||
hadoopConf: Configuration): Boolean = {
|
||||
/**
|
||||
* Add Spark Sql related table properties to the HIVE_TABLE_PROPERTIES.
|
||||
* @param sqlConf The spark sql conf.
|
||||
* @param schema The schema to write to the table.
|
||||
* @param parameters The origin parameters.
|
||||
* @return A new parameters added the HIVE_TABLE_PROPERTIES property.
|
||||
*/
|
||||
private def addSqlTableProperties(sqlConf: SQLConf, schema: StructType,
|
||||
parameters: Map[String, String]): Map[String, String] = {
|
||||
// Convert the schema and partition info used by spark sql to hive table properties.
|
||||
// The following code refers to the spark code in
|
||||
// https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
|
||||
|
||||
val partitionSet = parameters(HIVE_PARTITION_FIELDS_OPT_KEY)
|
||||
.split(",").map(_.trim).filter(!_.isEmpty).toSet
|
||||
val threshold = sqlConf.getConf(SCHEMA_STRING_LENGTH_THRESHOLD)
|
||||
|
||||
val (partitionCols, dataCols) = schema.partition(c => partitionSet.contains(c.name))
|
||||
val reOrderedType = StructType(dataCols ++ partitionCols)
|
||||
val schemaParts = reOrderedType.json.grouped(threshold).toSeq
|
||||
|
||||
var properties = Map(
|
||||
"spark.sql.sources.provider" -> "hudi",
|
||||
"spark.sql.sources.schema.numParts" -> schemaParts.size.toString
|
||||
)
|
||||
schemaParts.zipWithIndex.foreach { case (part, index) =>
|
||||
properties += s"spark.sql.sources.schema.part.$index" -> part
|
||||
}
|
||||
// add partition columns
|
||||
if (partitionSet.nonEmpty) {
|
||||
properties += "spark.sql.sources.schema.numPartCols" -> partitionSet.size.toString
|
||||
partitionSet.zipWithIndex.foreach { case (partCol, index) =>
|
||||
properties += s"spark.sql.sources.schema.partCol.$index" -> partCol
|
||||
}
|
||||
}
|
||||
var sqlPropertyText = ConfigUtils.configToString(properties)
|
||||
sqlPropertyText = if (parameters.containsKey(HIVE_TABLE_PROPERTIES)) {
|
||||
sqlPropertyText + "\n" + parameters(HIVE_TABLE_PROPERTIES)
|
||||
} else {
|
||||
sqlPropertyText
|
||||
}
|
||||
parameters + (HIVE_TABLE_PROPERTIES -> sqlPropertyText)
|
||||
}
|
||||
|
||||
private def createSqlTableSerdeProperties(parameters: Map[String, String], basePath: String): String = {
|
||||
val pathProp = s"path=$basePath"
|
||||
if (parameters.containsKey(HIVE_TABLE_SERDE_PROPERTIES)) {
|
||||
pathProp + "\n" + parameters(HIVE_TABLE_SERDE_PROPERTIES)
|
||||
} else {
|
||||
pathProp
|
||||
}
|
||||
}
|
||||
|
||||
private def metaSync(spark: SparkSession, parameters: Map[String, String], basePath: Path,
|
||||
schema: StructType): Boolean = {
|
||||
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
var metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
|
||||
parameters(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
|
||||
val newParameters = addSqlTableProperties(spark.sessionState.conf, schema, parameters)
|
||||
// for backward compatibility
|
||||
if (hiveSyncEnabled) {
|
||||
metaSyncEnabled = true
|
||||
@@ -435,17 +501,17 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
}
|
||||
var metaSyncSuccess = true
|
||||
if (metaSyncEnabled) {
|
||||
val fs = basePath.getFileSystem(hadoopConf)
|
||||
val fs = basePath.getFileSystem(spark.sessionState.newHadoopConf())
|
||||
syncClientToolClassSet.foreach(impl => {
|
||||
val syncSuccess = impl.trim match {
|
||||
case "org.apache.hudi.hive.HiveSyncTool" => {
|
||||
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
|
||||
syncHive(basePath, fs, parameters)
|
||||
syncHive(basePath, fs, newParameters)
|
||||
true
|
||||
}
|
||||
case _ => {
|
||||
val properties = new Properties();
|
||||
properties.putAll(parameters)
|
||||
properties.putAll(newParameters)
|
||||
properties.put("basePath", basePath.toString)
|
||||
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
|
||||
syncHoodie.syncHoodieTable()
|
||||
@@ -463,7 +529,9 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
*/
|
||||
case class TableInstantInfo(basePath: Path, instantTime: String, commitActionType: String, operation: WriteOperationType)
|
||||
|
||||
private def commitAndPerformPostOperations(writeResult: HoodieWriteResult,
|
||||
private def commitAndPerformPostOperations(spark: SparkSession,
|
||||
schema: StructType,
|
||||
writeResult: HoodieWriteResult,
|
||||
parameters: Map[String, String],
|
||||
client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
|
||||
tableConfig: HoodieTableConfig,
|
||||
@@ -497,7 +565,8 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
log.info(s"Compaction Scheduled is $compactionInstant")
|
||||
val metaSyncSuccess = metaSync(parameters, tableInstantInfo.basePath, jsc.hadoopConfiguration())
|
||||
|
||||
val metaSyncSuccess = metaSync(spark, parameters, tableInstantInfo.basePath, schema)
|
||||
|
||||
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
|
||||
if (!asyncCompactionEnabled) {
|
||||
|
||||
@@ -22,6 +22,7 @@ import java.util
|
||||
import java.util.{Collections, Date, UUID}
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap}
|
||||
import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
||||
@@ -29,10 +30,13 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.hive.HiveSyncConfig
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils}
|
||||
import org.apache.spark.SparkContext
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{Row, SQLContext, SaveMode, SparkSession}
|
||||
import org.mockito.ArgumentMatchers.any
|
||||
import org.mockito.Mockito.{spy, times, verify}
|
||||
@@ -486,6 +490,46 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
}
|
||||
})
|
||||
|
||||
test("Test build sync config for spark sql") {
|
||||
initSparkContext("test build sync config")
|
||||
val addSqlTablePropertiesMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("addSqlTableProperties",
|
||||
classOf[SQLConf], classOf[StructType], classOf[Map[_,_]])
|
||||
addSqlTablePropertiesMethod.setAccessible(true)
|
||||
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val basePath = "/tmp/hoodie_test"
|
||||
val params = Map(
|
||||
"path" -> basePath,
|
||||
DataSourceWriteOptions.TABLE_NAME_OPT_KEY -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY -> "partition"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val newParams = addSqlTablePropertiesMethod.invoke(HoodieSparkSqlWriter,
|
||||
spark.sessionState.conf, structType, parameters)
|
||||
.asInstanceOf[Map[String, String]]
|
||||
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[Map[_,_]])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(basePath), newParams).asInstanceOf[HiveSyncConfig]
|
||||
|
||||
assertResult("spark.sql.sources.provider=hudi\n" +
|
||||
"spark.sql.sources.schema.partCol.0=partition\n" +
|
||||
"spark.sql.sources.schema.numParts=1\n" +
|
||||
"spark.sql.sources.schema.numPartCols=1\n" +
|
||||
"spark.sql.sources.schema.part.0=" +
|
||||
"{\"type\":\"struct\",\"fields\":[{\"name\":\"_row_key\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," +
|
||||
"{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}," +
|
||||
"{\"name\":\"partition\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}")(hiveSyncConfig.tableProperties)
|
||||
|
||||
assertResult("path=/tmp/hoodie_test")(hiveSyncConfig.serdeProperties)
|
||||
}
|
||||
|
||||
case class Test(uuid: String, ts: Long)
|
||||
|
||||
import scala.collection.JavaConverters
|
||||
|
||||
Reference in New Issue
Block a user