1
0

[HUDI-3445] Support Clustering Command Based on Call Procedure Command for Spark SQL (#4901)

* [HUDI-3445] Clustering Command Based on Call Procedure Command for Spark SQL

* [HUDI-3445] Clustering Command Based on Call Procedure Command for Spark SQL

* [HUDI-3445] Clustering Command Based on Call Procedure Command for Spark SQL

Co-authored-by: shibei <huberylee.li@alibaba-inc.com>
This commit is contained in:
shibei
2022-03-04 09:33:16 +08:00
committed by GitHub
parent be9a264885
commit 62f534d002
20 changed files with 909 additions and 247 deletions

View File

@@ -18,12 +18,16 @@
package org.apache.hudi;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hadoop.fs.FileSystem;
@@ -80,4 +84,17 @@ public class HoodieDataSourceHelpers {
return metaClient.getCommitTimeline().filterCompletedInstants();
}
}
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public static Option<HoodieClusteringPlan> getClusteringPlan(FileSystem fs, String basePath, String instantTime) {
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf())
.setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
HoodieInstant hoodieInstant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime);
Option<Pair<HoodieInstant, HoodieClusteringPlan>> clusteringPlan = ClusteringUtils.getClusteringPlan(metaClient, hoodieInstant);
if (clusteringPlan.isPresent()) {
return Option.of(clusteringPlan.get().getValue());
} else {
return Option.empty();
}
}
}

View File

@@ -17,17 +17,15 @@
package org.apache.spark.sql.hudi.command
import org.apache.hudi.HoodieCommonUtils
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.{HoodieTimer, Option => HOption}
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation
import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.{Row, SparkSession}
@@ -44,19 +42,7 @@ case class CompactionHoodiePathCommand(path: String,
assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ,
s"Must compaction on a Merge On Read table.")
val schemaUtil = new TableSchemaResolver(metaClient)
val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(
HoodieSqlCommonUtils.withSparkConf(sparkSession, Map.empty)(
Map(
DataSourceWriteOptions.TABLE_TYPE.key() -> HoodieTableType.MERGE_ON_READ.name()
)
)
)
val jsc = new JavaSparkContext(sparkSession.sparkContext)
val client = DataSourceUtils.createHoodieClient(jsc, schemaStr, path,
metaClient.getTableConfig.getTableName, parameters)
val client = HoodieCommonUtils.createHoodieClientFromPath(sparkSession, path, Map.empty)
operation match {
case SCHEDULE =>

View File

@@ -21,12 +21,18 @@ import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.model.HoodieRecordPayload
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieClusteringException
import org.apache.hudi.index.HoodieIndex.IndexType
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.types._
import java.nio.charset.Charset
import java.sql.{Date, Timestamp}
abstract class BaseProcedure extends Procedure {
val INVALID_ARG_INDEX: Int = -1
@@ -68,14 +74,22 @@ abstract class BaseProcedure extends Procedure {
args.map.getOrDefault(key, INVALID_ARG_INDEX)
}
protected def getArgValueOrDefault(args: ProcedureArgs, parameter: ProcedureParameter): Any = {
protected def getArgValueOrDefault(args: ProcedureArgs, parameter: ProcedureParameter): Option[Any] = {
var argsIndex: Int = INVALID_ARG_INDEX
if (args.isNamedArgs) {
argsIndex = getArgsIndex(parameter.name, args)
} else {
argsIndex = getArgsIndex(parameter.index.toString, args)
}
if (argsIndex.equals(INVALID_ARG_INDEX)) parameter.default else getInternalRowValue(args.internalRow, argsIndex, parameter.dataType)
if (argsIndex.equals(INVALID_ARG_INDEX)) {
parameter.default match {
case option: Option[Any] => option
case _ => Option.apply(parameter.default)
}
} else {
Option.apply(getInternalRowValue(args.internalRow, argsIndex, parameter.dataType))
}
}
protected def getInternalRowValue(row: InternalRow, index: Int, dataType: DataType): Any = {
@@ -96,4 +110,40 @@ abstract class BaseProcedure extends Procedure {
throw new UnsupportedOperationException(s"type: ${dataType.typeName} not supported")
}
}
protected def getBasePath(tableName: Option[Any], tablePath: Option[Any]): String = {
tableName.map(
t => HoodieCatalogTable(sparkSession, new TableIdentifier(t.asInstanceOf[String])).tableLocation)
.getOrElse(
tablePath.map(p => p.asInstanceOf[String]).getOrElse(
throw new HoodieClusteringException("Table name or table path must be given one"))
)
}
protected def convertCatalystType(value: String, dataType: DataType): Any = {
try {
val valueWithType = dataType match {
case StringType => value
case BinaryType => value.getBytes(Charset.forName("utf-8"))
case BooleanType => value.toBoolean
case DoubleType => value.toDouble
case d: DecimalType => Decimal.apply(BigDecimal(value), d.precision, d.scale)
case FloatType => value.toFloat
case ByteType => value.toByte
case IntegerType => value.toInt
case LongType => value.toLong
case ShortType => value.toShort
case DateType => DateTimeUtils.fromJavaDate(Date.valueOf(value))
case TimestampType => DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(value))
case _ => throw new HoodieClusteringException("Data type not support:" + dataType)
}
valueWithType
} catch {
case e: HoodieClusteringException =>
throw e
case _ =>
throw new HoodieClusteringException("Data type not match, value:" + value + ", dataType:" + dataType)
}
}
}

View File

@@ -36,6 +36,8 @@ object HoodieProcedures {
mapBuilder.put(ShowCommitsProcedure.NAME, ShowCommitsProcedure.builder)
mapBuilder.put(ShowCommitsMetadataProcedure.NAME, ShowCommitsMetadataProcedure.builder)
mapBuilder.put(RollbackToInstantTimeProcedure.NAME, RollbackToInstantTimeProcedure.builder)
mapBuilder.put(RunClusteringProcedure.NAME, RunClusteringProcedure.builder)
mapBuilder.put(ShowClusteringProcedure.NAME, ShowClusteringProcedure.builder)
mapBuilder.build
}
}

View File

@@ -45,8 +45,8 @@ class RollbackToInstantTimeProcedure extends BaseProcedure with ProcedureBuilder
override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)
val table = getArgValueOrDefault(args, PARAMETERS(0)).asInstanceOf[String]
val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).asInstanceOf[String]
val table = getArgValueOrDefault(args, PARAMETERS(0)).get.asInstanceOf[String]
val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String]
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, new TableIdentifier(table))
val basePath = hoodieCatalogTable.tableLocation

View File

@@ -0,0 +1,176 @@
/*
* 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.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.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.{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.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.Expression
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 {
/**
* OPTIMIZE table_name|table_path [WHERE predicate]
* [ORDER BY (col_name1 [, ...] ) ]
*/
private val PARAMETERS = Array[ProcedureParameter](
ProcedureParameter.optional(0, "table", DataTypes.StringType, None),
ProcedureParameter.optional(1, "path", DataTypes.StringType, None),
ProcedureParameter.optional(2, "predicate", DataTypes.StringType, None),
ProcedureParameter.optional(3, "order", DataTypes.StringType, None)
)
private val OUTPUT_TYPE = new StructType(Array[StructField](
StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty),
StructField("partition", DataTypes.StringType, nullable = true, Metadata.empty),
StructField("groups", DataTypes.IntegerType, nullable = true, Metadata.empty)
))
def parameters: Array[ProcedureParameter] = PARAMETERS
def outputType: StructType = OUTPUT_TYPE
override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)
val tableName = getArgValueOrDefault(args, PARAMETERS(0))
val tablePath = getArgValueOrDefault(args, PARAMETERS(1))
val predicate = getArgValueOrDefault(args, PARAMETERS(2))
val orderColumns = getArgValueOrDefault(args, PARAMETERS(3))
val basePath: String = getBasePath(tableName, tablePath)
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build
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)
conf = conf ++ Map(
HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key() -> "SELECTED_PARTITIONS",
HoodieClusteringConfig.PARTITION_SELECTED.key() -> partitionSelected
)
logInfo(s"Partition predicates: ${p}, partition selected: ${partitionSelected}")
case _ =>
logInfo("No partition predicates")
}
// Construct sort column info
orderColumns match {
case Some(o) =>
validateOrderColumns(o.asInstanceOf[String], metaClient)
conf = conf ++ Map(
HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key() -> o.asInstanceOf[String]
)
logInfo(s"Order columns: ${o}")
case _ =>
logInfo("No order columns")
}
// Get all pending clustering instants
var pendingClustering = ClusteringUtils.getAllPendingClusteringPlans(metaClient)
.iterator().asScala.map(_.getLeft.getTimestamp).toSeq.sortBy(f => f)
logInfo(s"Pending clustering instants: ${pendingClustering.mkString(",")}")
val client = HoodieCommonUtils.createHoodieClientFromPath(sparkSession, basePath, conf)
val instantTime = HoodieActiveTimeline.createNewInstantTime
if (client.scheduleClusteringAtInstant(instantTime, HOption.empty())) {
pendingClustering ++= Seq(instantTime)
}
logInfo(s"Clustering instants to run: ${pendingClustering.mkString(",")}.")
val startTs = System.currentTimeMillis()
pendingClustering.foreach(client.cluster(_, true))
logInfo(s"Finish clustering all the instants: ${pendingClustering.mkString(",")}," +
s" time cost: ${System.currentTimeMillis() - startTs}ms.")
Seq.empty[Row]
}
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)
val partitionColumns = metaClient.getTableConfig.getPartitionFields.orElse(Array[String]())
// 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(",")
}
def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = {
if (orderColumns == null) {
throw new HoodieClusteringException("Order columns is null")
}
val tableSchemaResolver = new TableSchemaResolver(metaClient)
val fields = tableSchemaResolver.getTableAvroSchema(false)
.getFields.asScala.map(_.name().toLowerCase)
orderColumns.split(",").foreach(col => {
if (!fields.contains(col.toLowerCase)) {
throw new HoodieClusteringException("Order column not exist:" + col)
}
})
}
}
object RunClusteringProcedure {
val NAME = "run_clustering"
def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
override def get() = new RunClusteringProcedure
}
}

View File

@@ -0,0 +1,69 @@
/*
* 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.command.procedures
import org.apache.hudi.SparkAdapterSupport
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.util.ClusteringUtils
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
import java.util.function.Supplier
import scala.collection.JavaConverters._
class ShowClusteringProcedure extends BaseProcedure with ProcedureBuilder with SparkAdapterSupport with Logging {
private val PARAMETERS = Array[ProcedureParameter](
ProcedureParameter.optional(0, "table", DataTypes.StringType, None),
ProcedureParameter.optional(1, "path", DataTypes.StringType, None),
ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, 20)
)
private val OUTPUT_TYPE = new StructType(Array[StructField](
StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty),
StructField("groups", DataTypes.IntegerType, nullable = true, Metadata.empty)
))
def parameters: Array[ProcedureParameter] = PARAMETERS
def outputType: StructType = OUTPUT_TYPE
override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)
val tableName = getArgValueOrDefault(args, PARAMETERS(0))
val tablePath = getArgValueOrDefault(args, PARAMETERS(1))
val limit = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[Int]
val basePath: String = getBasePath(tableName, tablePath)
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build
ClusteringUtils.getAllPendingClusteringPlans(metaClient).iterator().asScala.map { p =>
Row(p.getLeft.getTimestamp, p.getRight.getInputGroups.size())
}.toSeq.take(limit)
}
override def build: Procedure = new ShowClusteringProcedure()
}
object ShowClusteringProcedure {
val NAME = "show_clustering"
def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
override def get() = new ShowClusteringProcedure
}
}

View File

@@ -75,8 +75,8 @@ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure
override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)
val table = getArgValueOrDefault(args, PARAMETERS(0)).asInstanceOf[String]
val limit = getArgValueOrDefault(args, PARAMETERS(1)).asInstanceOf[Int]
val table = getArgValueOrDefault(args, PARAMETERS(0)).get.asInstanceOf[String]
val limit = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[Int]
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, new TableIdentifier(table))
val basePath = hoodieCatalogTable.tableLocation

View File

@@ -61,20 +61,20 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface
CompactionTable(table, operation, timestamp)
}
override def visitCompactionOnPath (ctx: CompactionOnPathContext): LogicalPlan = withOrigin(ctx) {
override def visitCompactionOnPath(ctx: CompactionOnPathContext): LogicalPlan = withOrigin(ctx) {
val path = string(ctx.path)
val operation = CompactionOperation.withName(ctx.operation.getText.toUpperCase)
val timestamp = if (ctx.instantTimestamp != null) Some(ctx.instantTimestamp.getText.toLong) else None
CompactionPath(path, operation, timestamp)
}
override def visitShowCompactionOnTable (ctx: ShowCompactionOnTableContext): LogicalPlan = withOrigin(ctx) {
override def visitShowCompactionOnTable(ctx: ShowCompactionOnTableContext): LogicalPlan = withOrigin(ctx) {
val table = ctx.tableIdentifier().accept(this).asInstanceOf[LogicalPlan]
if (ctx.limit != null) {
CompactionShowOnTable(table, ctx.limit.getText.toInt)
} else {
CompactionShowOnTable(table)
}
if (ctx.limit != null) {
CompactionShowOnTable(table, ctx.limit.getText.toInt)
} else {
CompactionShowOnTable(table)
}
}
override def visitShowCompactionOnPath(ctx: ShowCompactionOnPathContext): LogicalPlan = withOrigin(ctx) {