1
0

[HUDI-3254] Introduce HoodieCatalog to manage tables for Spark Datasource V2 (#4611)

This commit is contained in:
leesf
2022-02-14 22:26:58 +08:00
committed by GitHub
parent 5ca4480a38
commit 0db1e978c6
26 changed files with 1288 additions and 81 deletions

View File

@@ -19,10 +19,8 @@ package org.apache.spark.sql.hudi
import org.apache.hudi.SparkAdapterSupport
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.expressions.{And, Cast, Expression, Literal}
import org.apache.spark.sql.catalyst.expressions.{And, Expression}
import org.apache.spark.sql.catalyst.plans.logical.{MergeIntoTable, SubqueryAlias}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, NullType}
object HoodieSqlUtils extends SparkAdapterSupport {
@@ -50,12 +48,4 @@ object HoodieSqlUtils extends SparkAdapterSupport {
case exp => Seq(exp)
}
}
def castIfNeeded(child: Expression, dataType: DataType, conf: SQLConf): Expression = {
child match {
case Literal(nul, NullType) => Literal(nul, dataType)
case _ => if (child.dataType != dataType)
Cast(child, dataType, Option(conf.sessionLocalTimeZone)) else child
}
}
}

View File

@@ -17,10 +17,10 @@
package org.apache.spark.sql.hudi.analysis
import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
import org.apache.hudi.DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.util.ReflectionUtils
import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, Literal, NamedExpression}
import org.apache.spark.sql.catalyst.plans.Inner
@@ -28,10 +28,10 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{getTableIdentifier, getTableLocation, isHoodieTable, removeMetaFields, tableExistsInPath}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{getTableIdentifier, removeMetaFields}
import org.apache.spark.sql.hudi.HoodieSqlUtils._
import org.apache.spark.sql.hudi.command._
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils, HoodieSqlUtils}
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.{AnalysisException, SparkSession}
@@ -42,12 +42,39 @@ object HoodieAnalysis {
Seq(
session => HoodieResolveReferences(session),
session => HoodieAnalysis(session)
)
) ++ extraResolutionRules()
def customPostHocResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
Seq(
session => HoodiePostAnalysisRule(session)
)
) ++ extraPostHocResolutionRules()
def extraResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] = {
if (!HoodieSparkUtils.beforeSpark3_2()) {
val spark3AnalysisClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3Analysis"
val spark3Analysis: SparkSession => Rule[LogicalPlan] =
session => ReflectionUtils.loadClass(spark3AnalysisClass, session).asInstanceOf[Rule[LogicalPlan]]
val spark3ResolveReferences = "org.apache.spark.sql.hudi.analysis.HoodieSpark3ResolveReferences"
val spark3References: SparkSession => Rule[LogicalPlan] =
session => ReflectionUtils.loadClass(spark3ResolveReferences, session).asInstanceOf[Rule[LogicalPlan]]
Seq(spark3Analysis, spark3References)
} else {
Seq.empty
}
}
def extraPostHocResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
if (!HoodieSparkUtils.beforeSpark3_2()) {
val spark3PostHocResolutionClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3PostAnalysisRule"
val spark3PostHocResolution: SparkSession => Rule[LogicalPlan] =
session => ReflectionUtils.loadClass(spark3PostHocResolutionClass, session).asInstanceOf[Rule[LogicalPlan]]
Seq(spark3PostHocResolution)
} else {
Seq.empty
}
}
/**
@@ -61,36 +88,36 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
plan match {
// Convert to MergeIntoHoodieTableCommand
case m @ MergeIntoTable(target, _, _, _, _)
if m.resolved && isHoodieTable(target, sparkSession) =>
if m.resolved && sparkAdapter.isHoodieTable(target, sparkSession) =>
MergeIntoHoodieTableCommand(m)
// Convert to UpdateHoodieTableCommand
case u @ UpdateTable(table, _, _)
if u.resolved && isHoodieTable(table, sparkSession) =>
if u.resolved && sparkAdapter.isHoodieTable(table, sparkSession) =>
UpdateHoodieTableCommand(u)
// Convert to DeleteHoodieTableCommand
case d @ DeleteFromTable(table, _)
if d.resolved && isHoodieTable(table, sparkSession) =>
if d.resolved && sparkAdapter.isHoodieTable(table, sparkSession) =>
DeleteHoodieTableCommand(d)
// Convert to InsertIntoHoodieTableCommand
case l if sparkAdapter.isInsertInto(l) =>
val (table, partition, query, overwrite, _) = sparkAdapter.getInsertIntoChildren(l).get
table match {
case relation: LogicalRelation if isHoodieTable(relation, sparkSession) =>
case relation: LogicalRelation if sparkAdapter.isHoodieTable(relation, sparkSession) =>
new InsertIntoHoodieTableCommand(relation, query, partition, overwrite)
case _ =>
l
}
// Convert to CreateHoodieTableAsSelectCommand
case CreateTable(table, mode, Some(query))
if query.resolved && isHoodieTable(table) =>
if query.resolved && sparkAdapter.isHoodieTable(table) =>
CreateHoodieTableAsSelectCommand(table, mode, query)
// Convert to CompactionHoodieTableCommand
case CompactionTable(table, operation, options)
if table.resolved && isHoodieTable(table, sparkSession) =>
if table.resolved && sparkAdapter.isHoodieTable(table, sparkSession) =>
val tableId = getTableIdentifier(table)
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableId)
CompactionHoodieTableCommand(catalogTable, operation, options)
@@ -99,7 +126,7 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
CompactionHoodiePathCommand(path, operation, options)
// Convert to CompactionShowOnTable
case CompactionShowOnTable(table, limit)
if isHoodieTable(table, sparkSession) =>
if sparkAdapter.isHoodieTable(table, sparkSession) =>
val tableId = getTableIdentifier(table)
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableId)
CompactionShowHoodieTableCommand(catalogTable, limit)
@@ -122,7 +149,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp {
// Resolve merge into
case mergeInto @ MergeIntoTable(target, source, mergeCondition, matchedActions, notMatchedActions)
if isHoodieTable(target, sparkSession) && target.resolved =>
if sparkAdapter.isHoodieTable(target, sparkSession) && target.resolved =>
val resolver = sparkSession.sessionState.conf.resolver
val resolvedSource = analyzer.execute(source)
@@ -277,7 +304,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
// Resolve update table
case UpdateTable(table, assignments, condition)
if isHoodieTable(table, sparkSession) && table.resolved =>
if sparkAdapter.isHoodieTable(table, sparkSession) && table.resolved =>
// Resolve condition
val resolvedCondition = condition.map(resolveExpressionFrom(table)(_))
// Resolve assignments
@@ -291,7 +318,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
// Resolve Delete Table
case DeleteFromTable(table, condition)
if isHoodieTable(table, sparkSession) && table.resolved =>
if sparkAdapter.isHoodieTable(table, sparkSession) && table.resolved =>
// Resolve condition
val resolvedCondition = condition.map(resolveExpressionFrom(table)(_))
// Return the resolved DeleteTable
@@ -303,7 +330,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
val (table, partition, query, overwrite, ifPartitionNotExists) =
sparkAdapter.getInsertIntoChildren(l).get
if (isHoodieTable(table, sparkSession) && query.resolved &&
if (sparkAdapter.isHoodieTable(table, sparkSession) && query.resolved &&
!containUnResolvedStar(query) &&
!checkAlreadyAppendMetaField(query)) {
val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala.map(
@@ -401,37 +428,37 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic
plan match {
// Rewrite the CreateDataSourceTableCommand to CreateHoodieTableCommand
case CreateDataSourceTableCommand(table, ignoreIfExists)
if isHoodieTable(table) =>
if sparkAdapter.isHoodieTable(table) =>
CreateHoodieTableCommand(table, ignoreIfExists)
// Rewrite the DropTableCommand to DropHoodieTableCommand
case DropTableCommand(tableName, ifExists, isView, purge)
if isHoodieTable(tableName, sparkSession) =>
if sparkAdapter.isHoodieTable(tableName, sparkSession) =>
DropHoodieTableCommand(tableName, ifExists, isView, purge)
// Rewrite the AlterTableDropPartitionCommand to AlterHoodieTableDropPartitionCommand
case AlterTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData)
if isHoodieTable(tableName, sparkSession) =>
if sparkAdapter.isHoodieTable(tableName, sparkSession) =>
AlterHoodieTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData)
// Rewrite the AlterTableRenameCommand to AlterHoodieTableRenameCommand
// Rewrite the AlterTableAddColumnsCommand to AlterHoodieTableAddColumnsCommand
case AlterTableAddColumnsCommand(tableId, colsToAdd)
if isHoodieTable(tableId, sparkSession) =>
if sparkAdapter.isHoodieTable(tableId, sparkSession) =>
AlterHoodieTableAddColumnsCommand(tableId, colsToAdd)
// Rewrite the AlterTableRenameCommand to AlterHoodieTableRenameCommand
case AlterTableRenameCommand(oldName, newName, isView)
if !isView && isHoodieTable(oldName, sparkSession) =>
if !isView && sparkAdapter.isHoodieTable(oldName, sparkSession) =>
new AlterHoodieTableRenameCommand(oldName, newName, isView)
// Rewrite the AlterTableChangeColumnCommand to AlterHoodieTableChangeColumnCommand
case AlterTableChangeColumnCommand(tableName, columnName, newColumn)
if isHoodieTable(tableName, sparkSession) =>
if sparkAdapter.isHoodieTable(tableName, sparkSession) =>
AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn)
// SPARK-34238: the definition of ShowPartitionsCommand has been changed in Spark3.2.
// Match the class type instead of call the `unapply` method.
case s: ShowPartitionsCommand
if isHoodieTable(s.tableName, sparkSession) =>
if sparkAdapter.isHoodieTable(s.tableName, sparkSession) =>
ShowHoodieTablePartitionsCommand(s.tableName, s.spec)
// Rewrite TruncateTableCommand to TruncateHoodieTableCommand
case TruncateTableCommand(tableName, partitionSpec)
if isHoodieTable(tableName, sparkSession) =>
if sparkAdapter.isHoodieTable(tableName, sparkSession) =>
new TruncateHoodieTableCommand(tableName, partitionSpec)
case _ => plan
}

View File

@@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Literal}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.hudi.HoodieSqlUtils.castIfNeeded
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}

View File

@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, BoundReference, Cast, EqualTo, Expression, Literal}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.hudi.HoodieSqlUtils.{castIfNeeded, getMergeIntoTargetTableId}
import org.apache.spark.sql.hudi.HoodieSqlUtils.getMergeIntoTargetTableId
import org.apache.spark.sql.hudi.SerDeUtils
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload._

View File

@@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression}
import org.apache.spark.sql.catalyst.plans.logical.{Assignment, UpdateTable}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.hudi.HoodieSqlUtils.castIfNeeded
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructField