1
0

[HUDI-3896] Porting Nested Schema Pruning optimization for Hudi's custom Relations (#5428)

Currently, all Hudi Relations bear performance gap relative to Spark's HadoopFsRelation 
and the reason to that is SchemaPruning optimization rule (pruning nested schemas) 
that is unfortunately predicated on usage of HadoopFsRelation, meaning that it's 
not applied in cases when any other relation is used.

This change is porting this rule to Hudi relations (MOR, Incremental, etc) 
by the virtue of leveraging HoodieSparkSessionExtensions mechanism 
injecting modified version of the original SchemaPruning rule 
that is adopted to work w/ Hudi's custom relations.

- Added customOptimizerRules to HoodieAnalysis
- Added NestedSchemaPrunning Spark's Optimizer rule
- Handle Spark's Optimizer pruned data schema (to effectively prune nested schemas)
- Enable HoodieClientTestHarness to inject HoodieSparkSessionExtensions
- Injecting Spark Session extensions for TestMORDataSource, TestCOWDataSource
- Disabled fallback to HadoopFsRelation
This commit is contained in:
Alexey Kudinkin
2022-07-21 02:36:06 -07:00
committed by GitHub
parent 2394c62973
commit de37774e12
42 changed files with 1220 additions and 500 deletions

View File

@@ -20,7 +20,7 @@ package org.apache.spark.sql
import HoodieSparkTypeUtils.isCastPreservingOrdering
import org.apache.spark.sql.catalyst.expressions.{Add, AttributeReference, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
object HoodieSpark3_2CatalystExpressionUtils extends HoodieCatalystExpressionUtils {
object HoodieSpark32CatalystExpressionUtils extends HoodieCatalystExpressionUtils {
override def tryMatchAttributeOrderingPreservingTransformation(expr: Expression): Option[AttributeReference] = {
expr match {

View File

@@ -0,0 +1,38 @@
/*
* 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
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, TimeTravelRelation}
object HoodieSpark32CatalystPlanUtils extends HoodieSpark3CatalystPlanUtils {
override def isRelationTimeTravel(plan: LogicalPlan): Boolean = {
plan.isInstanceOf[TimeTravelRelation]
}
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
plan match {
case timeTravel: TimeTravelRelation =>
Some((timeTravel.table, timeTravel.timestamp, timeTravel.version))
case _ =>
None
}
}
}

View File

@@ -19,47 +19,26 @@ package org.apache.spark.sql.adapter
import org.apache.avro.Schema
import org.apache.spark.sql.avro._
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.SPARK_VERSION
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetFileFormat}
import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_2CatalystExpressionUtils, SparkSession}
import org.apache.spark.sql._
/**
* Implementation of [[SparkAdapter]] for Spark 3.2.x branch
*/
class Spark3_2Adapter extends BaseSpark3Adapter {
def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark32CatalystPlanUtils
override def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializer =
new HoodieSpark3_2AvroSerializer(rootCatalystType, rootAvroType, nullable)
override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
new HoodieSpark3_2AvroDeserializer(rootAvroType, rootCatalystType)
override def createCatalystExpressionUtils(): HoodieCatalystExpressionUtils = HoodieSpark3_2CatalystExpressionUtils
/**
* if the logical plan is a TimeTravelRelation LogicalPlan.
*/
override def isRelationTimeTravel(plan: LogicalPlan): Boolean = {
plan.isInstanceOf[TimeTravelRelation]
}
/**
* Get the member of the TimeTravelRelation LogicalPlan.
*/
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
plan match {
case timeTravel: TimeTravelRelation =>
Some((timeTravel.table, timeTravel.timestamp, timeTravel.version))
case _ =>
None
}
}
override def getCatalystExpressionUtils(): HoodieCatalystExpressionUtils = HoodieSpark32CatalystExpressionUtils
override def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = {
Some(

View File

@@ -23,17 +23,17 @@ import org.apache.hudi.exception.HoodieException
import org.apache.hudi.sql.InsertMode
import org.apache.hudi.sync.common.util.ConfigUtils
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, SparkAdapterSupport}
import org.apache.spark.sql.HoodieSpark3SqlUtils.convertTransforms
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, HoodieCatalogTable}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange, UpdateColumnComment, UpdateColumnType}
import org.apache.spark.sql.connector.catalog._
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform}
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.hudi.analysis.HoodieV1OrV2Table
import org.apache.spark.sql.hudi.catalog.HoodieCatalog.convertTransforms
import org.apache.spark.sql.hudi.command._
import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, ProvidesHoodieConfig}
import org.apache.spark.sql.types.{StructField, StructType}
@@ -42,6 +42,7 @@ import org.apache.spark.sql.{Dataset, SaveMode, SparkSession, _}
import java.net.URI
import java.util
import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter}
import scala.collection.mutable
class HoodieCatalog extends DelegatingCatalogExtension
with StagingTableCatalog
@@ -343,3 +344,24 @@ class HoodieCatalog extends DelegatingCatalogExtension
})
}
}
object HoodieCatalog {
def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = {
val identityCols = new mutable.ArrayBuffer[String]
var bucketSpec = Option.empty[BucketSpec]
partitions.map {
case IdentityTransform(FieldReference(Seq(col))) =>
identityCols += col
case BucketTransform(numBuckets, FieldReference(Seq(col))) =>
bucketSpec = Some(BucketSpec(numBuckets, col :: Nil, Nil))
case _ =>
throw new HoodieException(s"Partitioning by expressions is not supported.")
}
(identityCols, bucketSpec)
}
}