1
0

[HUDI-4178] Addressing performance regressions in Spark DataSourceV2 Integration (#5737)

There are multiple issues with our current DataSource V2 integrations: b/c we advertise Hudi tables as V2, Spark expects it to implement certain APIs which are not implemented at the moment, instead we're using custom Resolution rule (in HoodieSpark3Analysis) to instead manually fallback to V1 APIs.  This commit fixes the issue by reverting DSv2 APIs and making Spark use V1, except for schema evaluation logic.
This commit is contained in:
Alexey Kudinkin
2022-06-07 16:30:46 -07:00
committed by GitHub
parent 1349b596a1
commit 35afdb4316
28 changed files with 374 additions and 256 deletions

View File

@@ -33,17 +33,13 @@ class HoodieSparkSessionExtension extends (SparkSessionExtensions => Unit)
new HoodieCommonSqlParser(session, parser)
}
HoodieAnalysis.customResolutionRules().foreach { rule =>
HoodieAnalysis.customResolutionRules.foreach { ruleBuilder =>
extensions.injectResolutionRule { session =>
rule(session)
ruleBuilder(session)
}
}
extensions.injectResolutionRule { session =>
sparkAdapter.createResolveHudiAlterTableCommand(session)
}
HoodieAnalysis.customPostHocResolutionRules().foreach { rule =>
HoodieAnalysis.customPostHocResolutionRules.foreach { rule =>
extensions.injectPostHocResolutionRule { session =>
rule(session)
}

View File

@@ -39,45 +39,69 @@ import org.apache.spark.sql.{AnalysisException, SparkSession}
import java.util
import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
object HoodieAnalysis {
def customResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
Seq(
type RuleBuilder = SparkSession => Rule[LogicalPlan]
def customResolutionRules: Seq[RuleBuilder] = {
val rules: ListBuffer[RuleBuilder] = ListBuffer(
// Default rules
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.gteqSpark3_2) {
val dataSourceV2ToV1FallbackClass = "org.apache.spark.sql.hudi.analysis.HoodieDataSourceV2ToV1Fallback"
val dataSourceV2ToV1Fallback: RuleBuilder =
session => ReflectionUtils.loadClass(dataSourceV2ToV1FallbackClass, session).asInstanceOf[Rule[LogicalPlan]]
val spark3AnalysisClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3Analysis"
val spark3Analysis: SparkSession => Rule[LogicalPlan] =
val spark3Analysis: RuleBuilder =
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]]
val spark3ResolveReferencesClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3ResolveReferences"
val spark3ResolveReferences: RuleBuilder =
session => ReflectionUtils.loadClass(spark3ResolveReferencesClass, session).asInstanceOf[Rule[LogicalPlan]]
Seq(spark3Analysis, spark3References)
} else {
Seq.empty
val spark32ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommandSpark32"
val spark32ResolveAlterTableCommands: RuleBuilder =
session => ReflectionUtils.loadClass(spark32ResolveAlterTableCommandsClass, 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)
} else if (HoodieSparkUtils.gteqSpark3_1) {
val spark31ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommand312"
val spark31ResolveAlterTableCommands: RuleBuilder =
session => ReflectionUtils.loadClass(spark31ResolveAlterTableCommandsClass, session).asInstanceOf[Rule[LogicalPlan]]
rules ++= Seq(spark31ResolveAlterTableCommands)
}
rules
}
def extraPostHocResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
def customPostHocResolutionRules: Seq[RuleBuilder] = {
val rules: ListBuffer[RuleBuilder] = ListBuffer(
// Default rules
session => HoodiePostAnalysisRule(session)
)
if (HoodieSparkUtils.gteqSpark3_2) {
val spark3PostHocResolutionClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3PostAnalysisRule"
val spark3PostHocResolution: SparkSession => Rule[LogicalPlan] =
val spark3PostHocResolution: RuleBuilder =
session => ReflectionUtils.loadClass(spark3PostHocResolutionClass, session).asInstanceOf[Rule[LogicalPlan]]
Seq(spark3PostHocResolution)
} else {
Seq.empty
rules += spark3PostHocResolution
}
rules
}
}
/**

View File

@@ -453,7 +453,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable)
// Enable the hive sync by default if spark have enable the hive metastore.
val enableHive = isEnableHive(sparkSession)
val enableHive = isUsingHiveCatalog(sparkSession)
withSparkConf(sparkSession, hoodieCatalogTable.catalogProperties) {
Map(
"path" -> path,