yihua commented on code in PR #5428:
URL: https://github.com/apache/hudi/pull/5428#discussion_r923975439
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -44,25 +44,24 @@ import org.apache.hudi.avro.HoodieAvroUtils
import scala.collection.JavaConverters._
-object HoodieSparkUtils extends SparkAdapterSupport {
-
- def isSpark2: Boolean = SPARK_VERSION.startsWith("2.")
-
- def isSpark3: Boolean = SPARK_VERSION.startsWith("3.")
-
- def isSpark3_0: Boolean = SPARK_VERSION.startsWith("3.0")
-
- def isSpark3_1: Boolean = SPARK_VERSION.startsWith("3.1")
-
- def gteqSpark3_1: Boolean = SPARK_VERSION > "3.1"
-
- def gteqSpark3_1_3: Boolean = SPARK_VERSION >= "3.1.3"
-
- def isSpark3_2: Boolean = SPARK_VERSION.startsWith("3.2")
+private[hudi] trait SparkVersionsSupport {
+ def getSparkVersion: String
+
+ def isSpark2: Boolean = getSparkVersion.startsWith("2.")
+ def isSpark3: Boolean = getSparkVersion.startsWith("3.")
+ def isSpark3_0: Boolean = getSparkVersion.startsWith("3.0")
+ def isSpark3_1: Boolean = getSparkVersion.startsWith("3.1")
+ def isSpark3_2: Boolean = getSparkVersion.startsWith("3.2")
+
+ def gteqSpark3_1: Boolean = getSparkVersion >= "3.1"
Review Comment:
I see the actual matching condition has changed. Is that intended, and is
the flag used before, which causes behavior change?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala:
##########
@@ -109,6 +110,11 @@ class DefaultSource extends RelationProvider
Option(schema)
}
+ // NOTE: We have to handle explicitly case of the Metadata Table (MT)
since by default all of Hudi
+ // relations will try to apply schema pruning techniques (like
nested schema pruning) which couldn't
+ // be applied to MT
+ val canPruneRelationSchema = !isMetadataTable(tablePath)
Review Comment:
Is this because MT uses HFile as the file format?
##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala:
##########
@@ -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 {
Review Comment:
Are these refactoring changes related to HoodieCatalystPlansUtils only
moving code around?
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -44,25 +44,24 @@ import org.apache.hudi.avro.HoodieAvroUtils
import scala.collection.JavaConverters._
-object HoodieSparkUtils extends SparkAdapterSupport {
-
- def isSpark2: Boolean = SPARK_VERSION.startsWith("2.")
-
- def isSpark3: Boolean = SPARK_VERSION.startsWith("3.")
-
- def isSpark3_0: Boolean = SPARK_VERSION.startsWith("3.0")
-
- def isSpark3_1: Boolean = SPARK_VERSION.startsWith("3.1")
-
- def gteqSpark3_1: Boolean = SPARK_VERSION > "3.1"
-
- def gteqSpark3_1_3: Boolean = SPARK_VERSION >= "3.1.3"
-
- def isSpark3_2: Boolean = SPARK_VERSION.startsWith("3.2")
+private[hudi] trait SparkVersionsSupport {
+ def getSparkVersion: String
+
+ def isSpark2: Boolean = getSparkVersion.startsWith("2.")
+ def isSpark3: Boolean = getSparkVersion.startsWith("3.")
+ def isSpark3_0: Boolean = getSparkVersion.startsWith("3.0")
+ def isSpark3_1: Boolean = getSparkVersion.startsWith("3.1")
+ def isSpark3_2: Boolean = getSparkVersion.startsWith("3.2")
+
+ def gteqSpark3_1: Boolean = getSparkVersion >= "3.1"
Review Comment:
Or it does not matter, because the spark version always has the patch
version (`x.y.z`).
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -127,36 +133,46 @@ abstract class HoodieBaseRelation(val sqlContext:
SQLContext,
* NOTE: Initialization of teh following members is coupled on purpose to
minimize amount of I/O
* required to fetch table's Avro and Internal schemas
*/
- protected lazy val (tableAvroSchema: Schema, internalSchema: InternalSchema)
= {
+ protected lazy val (tableAvroSchema: Schema, internalSchemaOpt:
Option[InternalSchema]) = {
val schemaResolver = new TableSchemaResolver(metaClient)
- val avroSchema: Schema = schemaSpec.map(convertToAvroSchema).getOrElse {
- Try(schemaResolver.getTableAvroSchema) match {
- case Success(schema) => schema
+ val internalSchemaOpt = if (!isSchemaEvolutionEnabled) {
+ None
+ } else {
+ Try(schemaResolver.getTableInternalSchemaFromCommitMetadata) match {
+ case Success(internalSchemaOpt) => toScalaOption(internalSchemaOpt)
case Failure(e) =>
- logError("Failed to fetch schema from the table", e)
- throw new HoodieSchemaException("Failed to fetch schema from the
table")
+ logWarning("Failed to fetch internal-schema from the table", e)
+ None
}
}
- val internalSchema: InternalSchema = if (!isSchemaEvolutionEnabled) {
- InternalSchema.getEmptyInternalSchema
- } else {
- Try(schemaResolver.getTableInternalSchemaFromCommitMetadata) match {
- case Success(internalSchemaOpt) =>
-
toScalaOption(internalSchemaOpt).getOrElse(InternalSchema.getEmptyInternalSchema)
+ val avroSchema = internalSchemaOpt.map { is =>
+ AvroInternalSchemaConverter.convert(is, "schema")
+ } orElse {
+ schemaSpec.map(convertToAvroSchema)
+ } getOrElse {
+ Try(schemaResolver.getTableAvroSchema) match {
+ case Success(schema) => schema
case Failure(e) =>
- logWarning("Failed to fetch internal-schema from the table", e)
- InternalSchema.getEmptyInternalSchema
+ logError("Failed to fetch schema from the table", e)
+ throw new HoodieSchemaException("Failed to fetch schema from the
table")
}
}
- (avroSchema, internalSchema)
+ (avroSchema, internalSchemaOpt)
}
- protected lazy val tableStructSchema: StructType =
AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
+ protected val tableStructSchema: StructType =
AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
Review Comment:
Should this be lazy?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala:
##########
@@ -187,23 +189,46 @@ private[sql] object SchemaConverters {
.values(toAvroType(vt, valueContainsNull, recordName, nameSpace))
case st: StructType =>
val childNameSpace = if (nameSpace != "") s"$nameSpace.$recordName"
else recordName
- val fieldsAssembler =
builder.record(recordName).namespace(nameSpace).fields()
- st.foreach { f =>
- val fieldAvroType =
- toAvroType(f.dataType, f.nullable, f.name, childNameSpace)
- fieldsAssembler.name(f.name).`type`(fieldAvroType).noDefault()
+ if (canBeUnion(st)) {
Review Comment:
Are these from Spark? How do we track any compatibility issues?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -227,9 +251,38 @@ abstract class HoodieBaseRelation(val sqlContext:
SQLContext,
/**
* Returns true in case table supports Schema on Read (Schema Evolution)
*/
- def hasSchemaOnRead: Boolean = !internalSchema.isEmptySchema
+ def hasSchemaOnRead: Boolean = internalSchemaOpt.isDefined
- override def schema: StructType = tableStructSchema
+ /**
+ * Data schema is determined as the actual schema of the Table's Data Files
(for ex, parquet/orc/etc);
+ *
+ * In cases when partition values are not persisted w/in the data files,
data-schema is defined as
+ * <pre>table's schema - partition columns</pre>
+ *
+ * Check scala-doc for [[shouldExtractPartitionValuesFromPartitionPath]] for
more details
+ */
+ def dataSchema: StructType =
+ if (shouldExtractPartitionValuesFromPartitionPath) {
+ prunePartitionColumns(tableStructSchema)
+ } else {
+ tableStructSchema
+ }
+
+ /**
+ * Determines whether relation's schema could be pruned by Spark's Optimizer
+ */
+ def canPruneRelationSchema: Boolean =
+ (fileFormat.isInstanceOf[ParquetFileFormat] ||
fileFormat.isInstanceOf[OrcFileFormat]) &&
Review Comment:
Should file format check automatically disable schema pruning for metadata
table?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -378,11 +433,12 @@ abstract class HoodieBaseRelation(val sqlContext:
SQLContext,
!SubqueryExpression.hasSubquery(condition)
}
- protected final def appendMandatoryRootFields(requestedColumns:
Array[String]): Array[String] = {
+ protected final def appendMandatoryColumns(requestedColumns: Array[String]):
Array[String] = {
// For a nested field in mandatory columns, we should first get the
root-level field, and then
// check for any missing column, as the requestedColumns should only
contain root-level fields
// We should only append root-level field as well
- val missing = mandatoryRootFields.filter(rootField =>
!requestedColumns.contains(rootField))
+ val missing = mandatoryFields.map(col =>
HoodieAvroUtils.getRootLevelFieldName(col))
+ .filter(rootField => !requestedColumns.contains(rootField))
Review Comment:
To clarify, here the top-level field (`x`) with all nested fields (`x.*.*`)
is kept even if only one nested field (`x.y.z`)under the top is needed because
of how Spark prunes the nested schema, right?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -514,32 +594,66 @@ abstract class HoodieBaseRelation(val sqlContext:
SQLContext,
// t/h Spark Session configuration (for ex, for Spark SQL)
optParams.getOrElse(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
||
- sparkSession.conf.get(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
-
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
+ sparkSession.conf.get(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
+
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
}
}
object HoodieBaseRelation extends SparkAdapterSupport {
private def generateUnsafeProjection(from: StructType, to: StructType) =
-
sparkAdapter.createCatalystExpressionUtils().generateUnsafeProjection(from, to)
+ sparkAdapter.getCatalystExpressionUtils().generateUnsafeProjection(from,
to)
def convertToAvroSchema(structSchema: StructType): Schema =
sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable =
false, "Record")
def getPartitionPath(fileStatus: FileStatus): Path =
fileStatus.getPath.getParent
+ /**
+ * Projects provided schema by picking only required (projected) top-level
columns from it
+ *
+ * @param tableSchema schema to project (either of [[InternalSchema]] or
Avro's [[Schema]])
+ * @param requiredColumns required top-level columns to be projected
+ */
+ def projectSchema(tableSchema: Either[Schema, InternalSchema],
requiredColumns: Array[String]): (Schema, StructType, InternalSchema) = {
Review Comment:
So this is our own projection by keeping top-level required columns. The
nested schema pruning is based on
`org.apache.spark.sql.execution.datasources.NestedSchemaPruning`.
##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/NestedSchemaPruning.scala:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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] {
Review Comment:
nit: add docs in this class to explain that?
##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/NestedSchemaPruning.scala:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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] {
Review Comment:
Do we expect the implementation to vary across Spark versions?
##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala:
##########
@@ -41,25 +41,38 @@ import java.util
import scala.collection.JavaConverters._
object HoodieAnalysis {
- def customResolutionRules(): Seq[SparkSession => Rule[LogicalPlan]] =
+
+ type RuleBuilder = SparkSession => Rule[LogicalPlan]
+
+ def customOptimizerRules(): Seq[RuleBuilder] =
+ if (HoodieSparkUtils.gteqSpark3_1) {
+ val nestedSchemaPruningClass =
"org.apache.spark.sql.execution.datasources.NestedSchemaPruning"
Review Comment:
It does support both Spark 3.1 and 3.2 after your changes, right?
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestNestedSchemaPruningOptimization.scala:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hudi
+
+import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{FileSourceScanExec, ProjectExec,
RowDataSourceScanExec, SparkPlan}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField,
StructType}
+import org.junit.jupiter.api.Assertions.assertEquals
+
+class TestNestedSchemaPruningOptimization extends HoodieSparkSqlTestBase with
SparkAdapterSupport {
Review Comment:
Have you run this manually with selective patch versions of Spark 2.4, 3.1,
and 3.2?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]