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

@@ -0,0 +1,77 @@
/*
* 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.hudi.spark3.internal.ReflectUtil
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.execution.command.ExplainCommand
import org.apache.spark.sql.execution.{ExtendedMode, SimpleMode}
abstract class HoodieSpark3CatalystPlanUtils extends HoodieCatalystPlansUtils {
def createExplainCommand(plan: LogicalPlan, extended: Boolean): LogicalPlan =
ExplainCommand(plan, mode = if (extended) ExtendedMode else SimpleMode)
override def toTableIdentifier(aliasId: AliasIdentifier): TableIdentifier = {
aliasId match {
case AliasIdentifier(name, Seq(database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq(_, database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq()) =>
TableIdentifier(name, None)
case _ => throw new IllegalArgumentException(s"Cannot cast $aliasId to TableIdentifier")
}
}
override def toTableIdentifier(relation: UnresolvedRelation): TableIdentifier = {
relation.multipartIdentifier.asTableIdentifier
}
override def createJoin(left: LogicalPlan, right: LogicalPlan, joinType: JoinType): Join = {
Join(left, right, joinType, None, JoinHint.NONE)
}
override def isInsertInto(plan: LogicalPlan): Boolean = {
plan.isInstanceOf[InsertIntoStatement]
}
override def getInsertIntoChildren(plan: LogicalPlan):
Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)] = {
plan match {
case insert: InsertIntoStatement =>
Some((insert.table, insert.partitionSpec, insert.query, insert.overwrite, insert.ifPartitionNotExists))
case _ =>
None
}
}
override def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]],
query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan = {
ReflectUtil.createInsertInto(table, partition, Seq.empty[String], query, overwrite, ifPartitionNotExists)
}
override def createLike(left: Expression, right: Expression): Expression = {
new Like(left, right)
}
}

View File

@@ -1,45 +0,0 @@
/*
* 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.hudi.exception.HoodieException
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform}
import scala.collection.mutable
object HoodieSpark3SqlUtils {
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)
}
}

View File

@@ -19,24 +19,20 @@ package org.apache.spark.sql.adapter
import org.apache.hudi.Spark3RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.spark3.internal.ReflectUtil
import org.apache.spark.SPARK_VERSION
import org.apache.spark.internal.Logging
import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Like, Predicate}
import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.{HoodieCatalystPlansUtils, HoodieSpark3CatalystPlanUtils, Row, SparkSession}
import scala.util.control.NonFatal
@@ -51,53 +47,10 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging {
override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters
override def toTableIdentifier(aliasId: AliasIdentifier): TableIdentifier = {
aliasId match {
case AliasIdentifier(name, Seq(database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq(_, database)) =>
TableIdentifier(name, Some(database))
case AliasIdentifier(name, Seq()) =>
TableIdentifier(name, None)
case _=> throw new IllegalArgumentException(s"Cannot cast $aliasId to TableIdentifier")
}
}
override def toTableIdentifier(relation: UnresolvedRelation): TableIdentifier = {
relation.multipartIdentifier.asTableIdentifier
}
override def createJoin(left: LogicalPlan, right: LogicalPlan, joinType: JoinType): Join = {
Join(left, right, joinType, None, JoinHint.NONE)
}
override def isInsertInto(plan: LogicalPlan): Boolean = {
plan.isInstanceOf[InsertIntoStatement]
}
override def getInsertIntoChildren(plan: LogicalPlan):
Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)] = {
plan match {
case insert: InsertIntoStatement =>
Some((insert.table, insert.partitionSpec, insert.query, insert.overwrite, insert.ifPartitionNotExists))
case _ =>
None
}
}
override def createInsertInto(table: LogicalPlan, partition: Map[String, Option[String]],
query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean): LogicalPlan = {
ReflectUtil.createInsertInto(table, partition, Seq.empty[String], query, overwrite, ifPartitionNotExists)
}
override def createSparkParsePartitionUtil(conf: SQLConf): SparkParsePartitionUtil = {
new Spark3ParsePartitionUtil(conf)
}
override def createLike(left: Expression, right: Expression): Expression = {
new Like(left, right)
}
override def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String] = {
parser.parseMultipartIdentifier(sqlText)
}
@@ -117,7 +70,7 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging {
case LogicalRelation(_, _, Some(table), _) => isHoodieTable(table)
case relation: UnresolvedRelation =>
try {
isHoodieTable(toTableIdentifier(relation), spark)
isHoodieTable(getCatalystPlanUtils.toTableIdentifier(relation), spark)
} catch {
case NonFatal(e) =>
logWarning("Failed to determine whether the table is a hoodie table", e)
@@ -128,19 +81,6 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging {
}
}
/**
* if the logical plan is a TimeTravelRelation LogicalPlan.
*/
override def isRelationTimeTravel(plan: LogicalPlan): Boolean = {
false
}
/**
* Get the member of the TimeTravelRelation LogicalPlan.
*/
override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark3.1.x")
}
override def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = {
// since spark3.2.1 support datasourceV2, so we need to a new SqlParser to deal DDL statment
if (SPARK_VERSION.startsWith("3.1")) {

View File

@@ -0,0 +1,197 @@
/*
* 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.execution.datasources
import org.apache.hudi.HoodieBaseRelation
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, Expression, NamedExpression, ProjectionOverSchema}
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
import org.apache.spark.sql.util.SchemaUtils.restoreOriginalOutputNames
/**
* Prunes unnecessary physical columns given a [[PhysicalOperation]] over a data source relation.
* By "physical column", we mean a column as defined in the data source format like Parquet format
* or ORC format. For example, in Spark SQL, a root-level Parquet column corresponds to a SQL
* column, and a nested Parquet column corresponds to a [[StructField]].
*
* NOTE: This class is borrowed from Spark 3.2.1, with modifications adapting it to handle [[HoodieBaseRelation]],
* instead of [[HadoopFsRelation]]
*/
class NestedSchemaPruning extends Rule[LogicalPlan] {
import org.apache.spark.sql.catalyst.expressions.SchemaPruning._
override def apply(plan: LogicalPlan): LogicalPlan =
if (conf.nestedSchemaPruningEnabled) {
apply0(plan)
} else {
plan
}
private def apply0(plan: LogicalPlan): LogicalPlan =
plan transformDown {
case op @ PhysicalOperation(projects, filters,
// NOTE: This is modified to accommodate for Hudi's custom relations, given that original
// [[NestedSchemaPruning]] rule is tightly coupled w/ [[HadoopFsRelation]]
// TODO generalize to any file-based relation
l @ LogicalRelation(relation: HoodieBaseRelation, _, _, _))
if relation.canPruneRelationSchema =>
prunePhysicalColumns(l.output, projects, filters, relation.dataSchema,
prunedDataSchema => {
val prunedRelation =
relation.updatePrunedDataSchema(prunedSchema = prunedDataSchema)
buildPrunedRelation(l, prunedRelation)
}).getOrElse(op)
}
/**
* This method returns optional logical plan. `None` is returned if no nested field is required or
* all nested fields are required.
*/
private def prunePhysicalColumns(output: Seq[AttributeReference],
projects: Seq[NamedExpression],
filters: Seq[Expression],
dataSchema: StructType,
outputRelationBuilder: StructType => LogicalRelation): Option[LogicalPlan] = {
val (normalizedProjects, normalizedFilters) =
normalizeAttributeRefNames(output, projects, filters)
val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters)
// If requestedRootFields includes a nested field, continue. Otherwise,
// return op
if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) {
val prunedDataSchema = pruneDataSchema(dataSchema, requestedRootFields)
// If the data schema is different from the pruned data schema, continue. Otherwise,
// return op. We effect this comparison by counting the number of "leaf" fields in
// each schemata, assuming the fields in prunedDataSchema are a subset of the fields
// in dataSchema.
if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
val prunedRelation = outputRelationBuilder(prunedDataSchema)
val projectionOverSchema = ProjectionOverSchema(prunedDataSchema)
Some(buildNewProjection(projects, normalizedProjects, normalizedFilters,
prunedRelation, projectionOverSchema))
} else {
None
}
} else {
None
}
}
/**
* Normalizes the names of the attribute references in the given projects and filters to reflect
* the names in the given logical relation. This makes it possible to compare attributes and
* fields by name. Returns a tuple with the normalized projects and filters, respectively.
*/
private def normalizeAttributeRefNames(output: Seq[AttributeReference],
projects: Seq[NamedExpression],
filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = {
val normalizedAttNameMap = output.map(att => (att.exprId, att.name)).toMap
val normalizedProjects = projects.map(_.transform {
case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) =>
att.withName(normalizedAttNameMap(att.exprId))
}).map { case expr: NamedExpression => expr }
val normalizedFilters = filters.map(_.transform {
case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) =>
att.withName(normalizedAttNameMap(att.exprId))
})
(normalizedProjects, normalizedFilters)
}
/**
* Builds the new output [[Project]] Spark SQL operator that has the `leafNode`.
*/
private def buildNewProjection(projects: Seq[NamedExpression],
normalizedProjects: Seq[NamedExpression],
filters: Seq[Expression],
prunedRelation: LogicalRelation,
projectionOverSchema: ProjectionOverSchema): Project = {
// Construct a new target for our projection by rewriting and
// including the original filters where available
val projectionChild =
if (filters.nonEmpty) {
val projectedFilters = filters.map(_.transformDown {
case projectionOverSchema(expr) => expr
})
val newFilterCondition = projectedFilters.reduce(And)
Filter(newFilterCondition, prunedRelation)
} else {
prunedRelation
}
// Construct the new projections of our Project by
// rewriting the original projections
val newProjects = normalizedProjects.map(_.transformDown {
case projectionOverSchema(expr) => expr
}).map { case expr: NamedExpression => expr }
if (log.isDebugEnabled) {
logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}")
}
Project(restoreOriginalOutputNames(newProjects, projects.map(_.name)), projectionChild)
}
/**
* Builds a pruned logical relation from the output of the output relation and the schema of the
* pruned base relation.
*/
private def buildPrunedRelation(outputRelation: LogicalRelation,
prunedBaseRelation: BaseRelation): LogicalRelation = {
val prunedOutput = getPrunedOutput(outputRelation.output, prunedBaseRelation.schema)
outputRelation.copy(relation = prunedBaseRelation, output = prunedOutput)
}
// Prune the given output to make it consistent with `requiredSchema`.
private def getPrunedOutput(output: Seq[AttributeReference],
requiredSchema: StructType): Seq[AttributeReference] = {
// We need to replace the expression ids of the pruned relation output attributes
// with the expression ids of the original relation output attributes so that
// references to the original relation's output are not broken
val outputIdMap = output.map(att => (att.name, att.exprId)).toMap
requiredSchema
.toAttributes
.map {
case att if outputIdMap.contains(att.name) =>
att.withExprId(outputIdMap(att.name))
case att => att
}
}
/**
* Counts the "leaf" fields of the given dataType. Informally, this is the
* number of fields of non-complex data type in the tree representation of
* [[DataType]].
*/
private def countLeaves(dataType: DataType): Int = {
dataType match {
case array: ArrayType => countLeaves(array.elementType)
case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType)
case struct: StructType =>
struct.map(field => countLeaves(field.dataType)).sum
case _ => 1
}
}
}