leesf commented on a change in pull request #2761:
URL: https://github.com/apache/hudi/pull/2761#discussion_r614772168



##########
File path: 
hudi-spark-datasource/hudi-spark3-extensions_2.12/src/main/scala/org/apache/hudi/execution/HudiSQLUtils.scala
##########
@@ -0,0 +1,553 @@
+/*
+ * 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.hudi.execution
+
+import java.io.File
+import java.util
+import java.util.{Locale, Properties}
+
+import com.google.common.cache.{CacheBuilder, CacheLoader}
+import org.apache.avro.generic.GenericRecord
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE_OPT_KEY, 
QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL}
+import org.apache.hudi.DataSourceWriteOptions._
+import org.apache.hudi.HoodieSparkSqlWriter.{TableInstantInfo, toProperties}
+import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, 
DataSourceUtils, DataSourceWriteOptions, DefaultSource, 
HoodieBootstrapPartition, HoodieBootstrapRelation, HoodieSparkSqlWriter, 
HoodieSparkUtils, HoodieWriterUtils, MergeOnReadSnapshotRelation}
+import 
org.apache.hudi.common.model.{OverwriteNonDefaultsWithLatestAvroPayload, 
OverwriteWithLatestAvroPayload}
+import org.apache.hudi.avro.HoodieAvroUtils
+import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient}
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.model._
+import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
+import org.apache.hudi.config.HoodieWriteConfig._
+import org.apache.hudi.exception.{HoodieException, HoodieIOException}
+import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
+import org.apache.hudi.payload.AWSDmsAvroPayload
+import org.apache.hudi.spark3.internal.HoodieDataSourceInternalTable
+import org.apache.log4j.LogManager
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, 
HiveTableRelation}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.functions.{col, udf}
+import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation}
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+
+
+/**
+  * hudi IUD utils
+  */
+object HudiSQLUtils {
+
+  val AWSDMSAVROPAYLOAD = classOf[AWSDmsAvroPayload].getName
+  val OVERWRITEWITHLATESTVROAYLOAD = 
classOf[OverwriteWithLatestAvroPayload].getName
+  val OVERWRITENONDEFAULTSWITHLATESTAVROPAYLOAD = 
classOf[OverwriteNonDefaultsWithLatestAvroPayload].getName
+  val MERGE_MARKER = "_hoodie_merge_marker"
+
+  private val log = LogManager.getLogger(getClass)
+
+  private val tableConfigCache = CacheBuilder
+    .newBuilder()
+    .maximumSize(1000)
+    .build(new CacheLoader[String, Properties] {
+      override def load(k: String): Properties = {
+        try {
+          
HoodieTableMetaClient.builder().setConf(SparkSession.active.sparkContext.hadoopConfiguration)
+            .setBasePath(k).build().getTableConfig.getProperties
+        } catch {
+          // we catch expected error here
+          case e: HoodieIOException =>
+            log.error(e.getMessage)
+            new Properties()
+          case t: Throwable =>
+            throw t
+        }
+      }
+    })
+
+  def getPropertiesFromTableConfigCache(path: String): Properties = {
+    if (path.isEmpty) {
+      throw new HoodieIOException("unexpected empty hoodie table basePath")
+    }
+    tableConfigCache.get(path)
+  }
+
+  private def matchHoodieRelation(relation: BaseRelation): Boolean = relation 
match {
+    case h: HadoopFsRelation if 
(h.options.get("hoodie.datasource.write.table.type") != null) => true
+    case m: MergeOnReadSnapshotRelation => true
+    case b: HoodieBootstrapRelation => true
+    case _ => false
+  }
+
+  def isHudiRelation(table: LogicalPlan): Boolean = {
+    table.collect {
+      case h: HiveTableRelation if 
(h.tableMeta.storage.inputFormat.getOrElse("").contains("Hoodie")) =>
+        true
+      case DataSourceV2Relation(_: HoodieDataSourceInternalTable, _, _, _, _) 
=> true
+      case LogicalRelation(r: BaseRelation, _, _, _) if 
(matchHoodieRelation(r)) => true
+    }.nonEmpty
+  }
+
+  def isHudiRelation(table: CatalogTable): Boolean = {
+    table.provider.map(_.toLowerCase(Locale.ROOT)).getOrElse("").equals("hudi")
+  }
+
+  def isHudiRelation(tableId: TableIdentifier, spark: SparkSession): Boolean = 
{
+    val table = spark.sessionState.catalog.getTableMetadata(tableId)
+    isHudiRelation(table)
+  }
+
+  def getTablePath(spark: SparkSession, table: CatalogTable): String = {
+    val url = if (table.tableType == CatalogTableType.MANAGED) {
+      Some(spark.sessionState.catalog.defaultTablePath(table.identifier))
+    } else {
+      table.storage.locationUri
+    }
+    val fs = new 
Path(url.get).getFileSystem(spark.sparkContext.hadoopConfiguration)
+    val rawPath = fs.makeQualified(new Path(url.get)).toUri.toString
+    // remove placeHolder
+    if (rawPath.endsWith("-PLACEHOLDER")) {
+      rawPath.substring(0, rawPath.length() - 16)
+    } else {
+      rawPath
+    }
+  }
+
+  def buildDefaultParameter(parameters: Map[String, String], enableHive: 
Boolean = true): Map[String, String] = {
+    val newParameters = 
HoodieWriterUtils.parametersWithWriteDefaults(parameters) ++ Map(
+      HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> 
"org.apache.hudi.hive.MultiPartKeysValueExtractor",
+      KEYGENERATOR_CLASS_PROP -> "org.apache.hudi.keygen.ComplexKeyGenerator",
+      HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> 
"org.apache.hudi.hive.MultiPartKeysValueExtractor")
+    val properties = new Properties()
+    properties.putAll(newParameters.asJava)
+    // correct partition info
+    if (properties.getProperty(PARTITIONPATH_FIELD_OPT_KEY, "").isEmpty) {
+      properties.put(KEYGENERATOR_CLASS_PROP, 
"org.apache.hudi.keygen.NonpartitionedKeyGenerator")
+    }
+    // set HIVE_STYLE_PARTITIONING_OPT_KEY to be true
+    properties.put(HIVE_STYLE_PARTITIONING_OPT_KEY, "true")
+
+    if (enableHive) {
+      properties.put(HIVE_SYNC_ENABLED_OPT_KEY, "true")
+      properties.put(HIVE_USE_JDBC_OPT_KEY, "false")
+      // correct hive partition info
+      if (properties.getProperty(PARTITIONPATH_FIELD_OPT_KEY, "").isEmpty) {
+        properties.put(HIVE_PARTITION_FIELDS_OPT_KEY, "")
+        properties.put(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, 
"org.apache.hudi.hive.NonPartitionedExtractor")
+      }
+    } else {
+      // disable Hive Sync
+      properties.put(HIVE_SYNC_ENABLED_OPT_KEY, "false")
+
+    }
+    properties.asScala.toMap
+  }
+
+  def getTablePathFromRelation(table: LogicalPlan, sparkSession: 
SparkSession): String = {
+    getHoodiePropsFromRelation(table, sparkSession).getOrElse("path", "")
+  }
+
+  /**
+    * build props form relation
+    */
+  def getHoodiePropsFromRelation(table: LogicalPlan, sparkSession: 
SparkSession): Map[String, String] = {
+    table.collect {
+      case h: HiveTableRelation =>
+        val table = h.asInstanceOf[HiveTableRelation].tableMeta
+        val db = table.identifier.database.getOrElse("default")
+        val rawTableName = table.identifier.table
+        val tableName = if (rawTableName.endsWith("_ro") || 
rawTableName.endsWith("_rt")) {
+          rawTableName.substring(0, rawTableName.size - 3)
+        } else {
+          rawTableName
+        }
+        val savePath = HudiSQLUtils.getTablePath(sparkSession, table)
+        table.properties ++ Map(HIVE_DATABASE_OPT_KEY -> db,
+          "inputformat" -> table.storage.inputFormat.getOrElse(""),
+          HIVE_TABLE_OPT_KEY -> tableName,
+          "path" -> savePath, "currentTable" -> table.identifier.quotedString) 
++ getPropertiesFromTableConfigCache(savePath).asScala.toMap
+      case l @ LogicalRelation(r: BaseRelation, _, _, _) =>
+        val catalogTable = l.catalogTable
+        val catalogProp = if (catalogTable.isEmpty) {
+          Map.empty[String, String]
+        } else {
+          val savePath = HudiSQLUtils.getTablePath(sparkSession, 
catalogTable.get)
+          val tablePath = if (savePath.endsWith("/*")) {
+            savePath.dropRight(catalogTable.get.partitionColumnNames.size*2 + 
1)
+          } else {
+            savePath
+          }
+          catalogTable.get.storage.properties ++ Map("currentTable" -> 
catalogTable.get.identifier.unquotedString,
+            "path" -> tablePath) ++ 
getPropertiesFromTableConfigCache(tablePath).asScala.toMap
+        }
+
+        val relationProp = r match {
+          case h: HadoopFsRelation => h.options.updated("path", 
h.options.getOrElse("hoodie.base.path", ""))
+          case m: MergeOnReadSnapshotRelation => m.optParams.updated("path", 
m.optParams.getOrElse("hoodie.base.path", ""))
+          case b: HoodieBootstrapRelation => b.optParams.updated("path", 
b.optParams.getOrElse("hoodie.base.path", ""))
+          case _ => Map.empty[String, String]
+        }
+        relationProp ++ catalogProp
+    }.headOption.getOrElse(Map.empty[String, String])
+  }
+
+  def getRequredFieldsFromTableConf(tablePath: String): (String, Seq[String]) 
= {
+    val tableConf = tableConfigCache.get(tablePath)
+    val recordKeyFields = 
tableConf.getProperty(RECORDKEY_FIELD_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty)
+    val partitionPathFields = 
tableConf.getProperty(PARTITIONPATH_FIELD_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty)
+    val preCombineField = tableConf.getProperty(PRECOMBINE_FIELD_OPT_KEY).trim
+    (preCombineField, recordKeyFields ++ partitionPathFields)
+  }
+
+  def merge(
+      df: DataFrame,
+      spark: SparkSession,
+      tableMeta: Map[String, String],
+      trySkipIndex: Boolean = true,
+      enableHive: Boolean = true): Unit = {
+    val savePath = tableMeta.getOrElse("path", throw new 
HoodieException("cannot find table Path, pls check your hoodie table!!!"))
+    val payload = 
HudiSQLUtils.tableConfigCache.get(savePath).getProperty(PAYLOAD_CLASS_OPT_KEY, 
DEFAULT_PAYLOAD_OPT_VAL)
+
+    val df2Merge = payload match {
+      case AWSDMSAVROPAYLOAD =>
+        df.drop("Op").withColumnRenamed(MERGE_MARKER, "Op")
+      case _ =>
+        val f: (String) => Boolean = {x => {if (x == "D") true else false}}
+        val markDeleteUdf = udf(f)
+        df.withColumn(MERGE_MARKER, markDeleteUdf(col(MERGE_MARKER)))
+    }
+    update(df2Merge, buildDefaultParameter(tableMeta, enableHive), spark, 
WriteOperationType.UPSERT, trySkipIndex, enableHive)
+  }
+
+  def update(
+      df: DataFrame,
+      parameters: Map[String, String],
+      spark: SparkSession,
+      writeOperationType: WriteOperationType = WriteOperationType.UPSERT,
+      trySkipIndex: Boolean = true, enableHive: Boolean = true): Unit = {
+    val tablePath = parameters.get("path").get
+
+    val tableConfig = tableConfigCache.get(tablePath)
+    tableConfig.put(TABLE_TYPE_OPT_KEY, 
tableConfig.getProperty("hoodie.table.type"))
+    val tblName = tableConfig.get("hoodie.table.name").toString
+    tableConfig.put(HIVE_TABLE_OPT_KEY, tblName)
+    val dataWriteOptions = parameters ++ addSetProperties(parameters, spark) 
++ tableConfig.asScala
+
+    checkWriteOptions(dataWriteOptions, enableHive)
+
+    if (df.schema.exists(f => 
f.name.endsWith(HoodieRecord.FILENAME_METADATA_FIELD)) && trySkipIndex) {
+      log.info("try to upsert table skip index")
+      val par = dataWriteOptions.getOrElse("par", "1500")
+      upsertSkipIndex(df, dataWriteOptions ++ 
Map("hoodie.tagging.before.insert" -> "false",

Review comment:
       hoodie.tagging.before.insert




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to