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

@@ -41,19 +41,6 @@ class Spark3_1Adapter extends BaseSpark3Adapter {
override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
new HoodieSpark3_1AvroDeserializer(rootAvroType, rootCatalystType)
override def createResolveHudiAlterTableCommand(sparkSession: SparkSession): Rule[LogicalPlan] = {
if (SPARK_VERSION.startsWith("3.1")) {
val loadClassName = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommand312"
val clazz = Class.forName(loadClassName, true, Thread.currentThread().getContextClassLoader)
val ctor = clazz.getConstructors.head
ctor.newInstance(sparkSession).asInstanceOf[Rule[LogicalPlan]]
} else {
new Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan
}
}
}
override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = {
Some(new Spark31HoodieParquetFileFormat(appendPartitionValues))
}

View File

@@ -16,6 +16,8 @@
*/
package org.apache.spark.sql.hudi
import org.apache.hudi.common.config.HoodieCommonConfig
import java.util.Locale
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
@@ -114,8 +116,9 @@ case class ResolveHudiAlterTableCommand312(sparkSession: SparkSession) extends R
}
}
private def schemaEvolutionEnabled(): Boolean = sparkSession
.sessionState.conf.getConfString(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean
private def schemaEvolutionEnabled(): Boolean =
sparkSession.sessionState.conf.getConfString(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key,
HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.defaultValue.toString).toBoolean
private def isHoodieTable(table: CatalogTable): Boolean = table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"