[HUDI-4186] Support Hudi with Spark 3.3.0 (#5943)
Co-authored-by: Shawn Chang <yxchang@amazon.com>
This commit is contained in:
@@ -44,15 +44,24 @@ import scala.collection.mutable.ListBuffer
|
||||
object HoodieAnalysis {
|
||||
type RuleBuilder = SparkSession => Rule[LogicalPlan]
|
||||
|
||||
def customOptimizerRules: Seq[RuleBuilder] =
|
||||
def customOptimizerRules: Seq[RuleBuilder] = {
|
||||
if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
val nestedSchemaPruningClass = "org.apache.spark.sql.execution.datasources.NestedSchemaPruning"
|
||||
val nestedSchemaPruningRule = ReflectionUtils.loadClass(nestedSchemaPruningClass).asInstanceOf[Rule[LogicalPlan]]
|
||||
val nestedSchemaPruningClass =
|
||||
if (HoodieSparkUtils.gteqSpark3_3) {
|
||||
"org.apache.spark.sql.execution.datasources.Spark33NestedSchemaPruning"
|
||||
} else if (HoodieSparkUtils.gteqSpark3_2) {
|
||||
"org.apache.spark.sql.execution.datasources.Spark32NestedSchemaPruning"
|
||||
} else {
|
||||
// spark 3.1
|
||||
"org.apache.spark.sql.execution.datasources.Spark31NestedSchemaPruning"
|
||||
}
|
||||
|
||||
val nestedSchemaPruningRule = ReflectionUtils.loadClass(nestedSchemaPruningClass).asInstanceOf[Rule[LogicalPlan]]
|
||||
Seq(_ => nestedSchemaPruningRule)
|
||||
} else {
|
||||
Seq.empty
|
||||
}
|
||||
}
|
||||
|
||||
def customResolutionRules: Seq[RuleBuilder] = {
|
||||
val rules: ListBuffer[RuleBuilder] = ListBuffer(
|
||||
@@ -74,18 +83,21 @@ object HoodieAnalysis {
|
||||
val spark3ResolveReferences: RuleBuilder =
|
||||
session => ReflectionUtils.loadClass(spark3ResolveReferencesClass, session).asInstanceOf[Rule[LogicalPlan]]
|
||||
|
||||
val spark32ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommandSpark32"
|
||||
val spark32ResolveAlterTableCommands: RuleBuilder =
|
||||
session => ReflectionUtils.loadClass(spark32ResolveAlterTableCommandsClass, session).asInstanceOf[Rule[LogicalPlan]]
|
||||
val resolveAlterTableCommandsClass =
|
||||
if (HoodieSparkUtils.gteqSpark3_3)
|
||||
"org.apache.spark.sql.hudi.Spark33ResolveHudiAlterTableCommand"
|
||||
else "org.apache.spark.sql.hudi.Spark32ResolveHudiAlterTableCommand"
|
||||
val resolveAlterTableCommands: RuleBuilder =
|
||||
session => ReflectionUtils.loadClass(resolveAlterTableCommandsClass, session).asInstanceOf[Rule[LogicalPlan]]
|
||||
|
||||
// NOTE: PLEASE READ CAREFULLY
|
||||
//
|
||||
// It's critical for this rules to follow in this order, so that DataSource V2 to V1 fallback
|
||||
// is performed prior to other rules being evaluated
|
||||
rules ++= Seq(dataSourceV2ToV1Fallback, spark3Analysis, spark3ResolveReferences, spark32ResolveAlterTableCommands)
|
||||
rules ++= Seq(dataSourceV2ToV1Fallback, spark3Analysis, spark3ResolveReferences, resolveAlterTableCommands)
|
||||
|
||||
} else if (HoodieSparkUtils.gteqSpark3_1) {
|
||||
val spark31ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommand312"
|
||||
val spark31ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.Spark312ResolveHudiAlterTableCommand"
|
||||
val spark31ResolveAlterTableCommands: RuleBuilder =
|
||||
session => ReflectionUtils.loadClass(spark31ResolveAlterTableCommandsClass, session).asInstanceOf[Rule[LogicalPlan]]
|
||||
|
||||
@@ -421,12 +433,10 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
|
||||
UpdateTable(table, resolvedAssignments, resolvedCondition)
|
||||
|
||||
// Resolve Delete Table
|
||||
case DeleteFromTable(table, condition)
|
||||
case dft @ DeleteFromTable(table, condition)
|
||||
if sparkAdapter.isHoodieTable(table, sparkSession) && table.resolved =>
|
||||
// Resolve condition
|
||||
val resolvedCondition = condition.map(resolveExpressionFrom(table)(_))
|
||||
// Return the resolved DeleteTable
|
||||
DeleteFromTable(table, resolvedCondition)
|
||||
val resolveExpression = resolveExpressionFrom(table, None)_
|
||||
sparkAdapter.resolveDeleteFromTable(dft, resolveExpression)
|
||||
|
||||
// Append the meta field to the insert query to walk through the validate for the
|
||||
// number of insert fields with the number of the target table fields.
|
||||
|
||||
@@ -21,6 +21,7 @@ import org.apache.hudi.SparkAdapterSupport
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
|
||||
import org.apache.spark.sql.hudi.ProvidesHoodieConfig
|
||||
|
||||
@@ -36,9 +37,9 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Hoodie
|
||||
|
||||
// Remove meta fields from the data frame
|
||||
var df = removeMetaFields(Dataset.ofRows(sparkSession, table))
|
||||
if (deleteTable.condition.isDefined) {
|
||||
df = df.filter(Column(deleteTable.condition.get))
|
||||
}
|
||||
// SPARK-38626 DeleteFromTable.condition is changed from Option[Expression] to Expression in Spark 3.3
|
||||
val condition = sparkAdapter.extractCondition(deleteTable)
|
||||
if (condition != null) df = df.filter(Column(condition))
|
||||
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableId)
|
||||
val config = buildHoodieDeleteTableConfig(hoodieCatalogTable, sparkSession)
|
||||
|
||||
@@ -57,6 +57,14 @@ class HoodieCommonSqlParser(session: SparkSession, delegate: ParserInterface)
|
||||
|
||||
override def parseDataType(sqlText: String): DataType = delegate.parseDataType(sqlText)
|
||||
|
||||
/* SPARK-37266 Added parseQuery to ParserInterface in Spark 3.3.0. This is a patch to prevent
|
||||
hackers from tampering text with persistent view, it won't be called in older Spark
|
||||
Don't mark this as override for backward compatibility
|
||||
Can't use sparkExtendedParser directly here due to the same reason */
|
||||
def parseQuery(sqlText: String): LogicalPlan = parse(sqlText) { parser =>
|
||||
sparkAdapter.getQueryParserFromExtendedSqlParser(session, delegate, sqlText)
|
||||
}
|
||||
|
||||
def parseRawDataType(sqlText : String) : DataType = {
|
||||
throw new UnsupportedOperationException(s"Unsupported parseRawDataType method")
|
||||
}
|
||||
|
||||
@@ -139,9 +139,8 @@ class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll {
|
||||
try {
|
||||
spark.sql(sql)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
assertResult(true)(e.getMessage.contains(errorMsg))
|
||||
hasException = true
|
||||
case e: Throwable if e.getMessage.contains(errorMsg) => hasException = true
|
||||
case f: Throwable => fail("Exception should contain: " + errorMsg + ", error message: " + f.getMessage, f)
|
||||
}
|
||||
assertResult(true)(hasException)
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.HoodieSparkUtils
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator}
|
||||
@@ -210,8 +211,14 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase {
|
||||
spark.sql(s"""insert into $tableName values (1, "z3", "v1", "2021-10-01"), (2, "l4", "v1", "2021-10-02")""")
|
||||
|
||||
// specify duplicate partition columns
|
||||
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01', dt='2021-10-02')")(
|
||||
"Found duplicate keys 'dt'")
|
||||
if (HoodieSparkUtils.gteqSpark3_3) {
|
||||
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01', dt='2021-10-02')")(
|
||||
"Found duplicate keys `dt`")
|
||||
} else {
|
||||
checkExceptionContain(s"alter table $tableName drop partition (dt='2021-10-01', dt='2021-10-02')")(
|
||||
"Found duplicate keys 'dt'")
|
||||
}
|
||||
|
||||
|
||||
// drop 2021-10-01 partition
|
||||
spark.sql(s"alter table $tableName drop partition (dt='2021-10-01')")
|
||||
|
||||
@@ -31,6 +31,7 @@ class TestNestedSchemaPruningOptimization extends HoodieSparkSqlTestBase with Sp
|
||||
val explainCommand = sparkAdapter.getCatalystPlanUtils.createExplainCommand(plan, extended = true)
|
||||
executePlan(explainCommand)
|
||||
.executeCollect()
|
||||
.map(_.getString(0))
|
||||
.mkString("\n")
|
||||
}
|
||||
|
||||
|
||||
@@ -82,7 +82,11 @@ class TestCallCommandParser extends HoodieSparkSqlTestBase {
|
||||
}
|
||||
|
||||
test("Test Call Parse Error") {
|
||||
checkParseExceptionContain("CALL cat.system radish kebab")("mismatched input 'CALL' expecting")
|
||||
if (HoodieSparkUtils.gteqSpark3_3) {
|
||||
checkParseExceptionContain("CALL cat.system radish kebab")("Syntax error at or near 'CALL'")
|
||||
} else {
|
||||
checkParseExceptionContain("CALL cat.system radish kebab")("mismatched input 'CALL' expecting")
|
||||
}
|
||||
}
|
||||
|
||||
test("Test Call Produce with semicolon") {
|
||||
@@ -110,9 +114,8 @@ class TestCallCommandParser extends HoodieSparkSqlTestBase {
|
||||
try {
|
||||
parser.parsePlan(sql)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
assertResult(true)(e.getMessage.contains(errorMsg))
|
||||
hasException = true
|
||||
case e: Throwable if e.getMessage.contains(errorMsg) => hasException = true
|
||||
case f: Throwable => fail("Exception should contain: " + errorMsg + ", error message: " + f.getMessage, f)
|
||||
}
|
||||
assertResult(true)(hasException)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user