1
0

[HUDI-3567] Refactor HoodieCommonUtils to make code more reasonable (#4982)

This commit is contained in:
huberylee
2022-03-12 05:23:19 +08:00
committed by GitHub
parent b00180342e
commit 56cb49485d
11 changed files with 706 additions and 593 deletions

View File

@@ -17,7 +17,7 @@
package org.apache.spark.sql.hudi.command
import org.apache.hudi.HoodieCommonUtils
import org.apache.hudi.HoodieCLIUtils
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
@@ -42,7 +42,7 @@ case class CompactionHoodiePathCommand(path: String,
assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ,
s"Must compaction on a Merge On Read table.")
val client = HoodieCommonUtils.createHoodieClientFromPath(sparkSession, path, Map.empty)
val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, path, Map.empty)
operation match {
case SCHEDULE =>

View File

@@ -17,29 +17,24 @@
package org.apache.spark.sql.hudi.command.procedures
import org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.ValidationUtils.checkArgument
import org.apache.hudi.common.util.{ClusteringUtils, Option => HOption}
import org.apache.hudi.config.HoodieClusteringConfig
import org.apache.hudi.exception.HoodieClusteringException
import org.apache.hudi.metadata.HoodieTableMetadata
import org.apache.hudi.{HoodieCommonUtils, SparkAdapterSupport}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.hudi.{AvroConversionUtils, HoodieCLIUtils, HoodieFileIndex}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row}
import org.apache.spark.sql.catalyst.expressions.PredicateHelper
import org.apache.spark.sql.execution.datasources.FileStatusCache
import org.apache.spark.sql.types._
import java.util.Properties
import java.util.function.Supplier
import scala.collection.JavaConverters
import scala.collection.JavaConverters._
class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with SparkAdapterSupport with Logging {
class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with PredicateHelper with Logging {
/**
* OPTIMIZE table_name|table_path [WHERE predicate]
* [ORDER BY (col_name1 [, ...] ) ]
@@ -74,15 +69,12 @@ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with Sp
var conf: Map[String, String] = Map.empty
predicate match {
case Some(p) =>
val partitionColumnsSchema = HoodieCommonUtils.getPartitionSchemaFromProperty(metaClient, None)
val partitionPredicate = HoodieCommonUtils.resolveFilterExpr(
spark, p.asInstanceOf[String], partitionColumnsSchema)
val partitionSelected = prunePartition(metaClient, partitionPredicate)
val prunedPartitions = prunePartition(metaClient, p.asInstanceOf[String])
conf = conf ++ Map(
HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key() -> "SELECTED_PARTITIONS",
HoodieClusteringConfig.PARTITION_SELECTED.key() -> partitionSelected
HoodieClusteringConfig.PARTITION_SELECTED.key() -> prunedPartitions
)
logInfo(s"Partition predicates: ${p}, partition selected: ${partitionSelected}")
logInfo(s"Partition predicates: ${p}, partition selected: ${prunedPartitions}")
case _ =>
logInfo("No partition predicates")
}
@@ -104,7 +96,7 @@ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with Sp
.iterator().asScala.map(_.getLeft.getTimestamp).toSeq.sortBy(f => f)
logInfo(s"Pending clustering instants: ${pendingClustering.mkString(",")}")
val client = HoodieCommonUtils.createHoodieClientFromPath(sparkSession, basePath, conf)
val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, basePath, conf)
val instantTime = HoodieActiveTimeline.createNewInstantTime
if (client.scheduleClusteringAtInstant(instantTime, HOption.empty())) {
pendingClustering ++= Seq(instantTime)
@@ -120,37 +112,26 @@ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with Sp
override def build: Procedure = new RunClusteringProcedure()
def prunePartition(metaClient: HoodieTableMetaClient, partitionPredicate: Expression): String = {
val partitionSchema = HoodieCommonUtils.getPartitionSchemaFromProperty(metaClient, None)
// Get tableName meta data
val engineContext = new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext))
val properties = new Properties()
properties.putAll(JavaConverters.mapAsJavaMapConverter(sparkSession.sessionState.conf.getAllConfs).asJava)
val metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(properties).build()
val tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath,
FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue)
val sparkParsePartitionUtil = sparkAdapter.createSparkParsePartitionUtil(sparkSession.sessionState.conf)
val typedProperties = HoodieCommonUtils.getConfigProperties(sparkSession, Map.empty)
def prunePartition(metaClient: HoodieTableMetaClient, predicate: String): String = {
val options = Map(QUERY_TYPE.key() -> QUERY_TYPE_SNAPSHOT_OPT_VAL, "path" -> metaClient.getBasePath)
val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, None, options,
FileStatusCache.getOrCreate(sparkSession))
// Resolve partition predicates
val schemaResolver = new TableSchemaResolver(metaClient)
val tableSchema = AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema)
val condition = HoodieCatalystExpressionUtils.resolveFilterExpr(sparkSession, predicate, tableSchema)
val partitionColumns = metaClient.getTableConfig.getPartitionFields.orElse(Array[String]())
val (partitionPredicates, dataPredicates) = HoodieCatalystExpressionUtils.splitPartitionAndDataPredicates(
sparkSession, splitConjunctivePredicates(condition).toArray, partitionColumns)
checkArgument(dataPredicates.isEmpty, "Only partition predicates are allowed")
// Translate all partition path to {@code org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath}
val partitionPaths = tableMetadata.getAllPartitionPaths.asScala.map(partitionPath => {
val partitionColumnValues = HoodieCommonUtils.parsePartitionColumnValues(
sparkParsePartitionUtil, typedProperties, metaClient.getBasePath,
partitionSchema, partitionColumns, partitionPath)
new PartitionPath(partitionPath, partitionColumnValues)
})
// Filter partition by predicates
val selectedPartitions = HoodieCommonUtils.prunePartition(
partitionSchema, partitionPaths, partitionPredicate)
selectedPartitions.map(partitionPath => partitionPath.getPath).toSet.mkString(",")
// Get all partitions and prune partition by predicates
val prunedPartitions = hoodieFileIndex.getPartitionPaths(partitionPredicates)
prunedPartitions.map(partitionPath => partitionPath.getPath).toSet.mkString(",")
}
def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = {
private def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = {
if (orderColumns == null) {
throw new HoodieClusteringException("Order columns is null")
}