alexeykudinkin commented on code in PR #5428:
URL: https://github.com/apache/hudi/pull/5428#discussion_r897431574
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -74,11 +79,13 @@ case class HoodieTableState(tablePath: String,
abstract class HoodieBaseRelation(val sqlContext: SQLContext,
val metaClient: HoodieTableMetaClient,
val optParams: Map[String, String],
- userSchema: Option[StructType])
+ userSchema: Option[StructType],
+ val canPruneSchema: Boolean = true)
extends BaseRelation
with FileRelation
with PrunedFilteredScan
- with Logging {
+ with Logging
+ with SparkAdapterSupport {
Review Comment:
This is already the case for Spark 3.2 (Spark SQL won't work) w/o those
extensions, here we'd just need to extend it to Spark 3.1 as well
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala:
##########
@@ -88,7 +86,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
options = optParams,
// NOTE: We have to fork the Hadoop Config here as Spark will be
modifying it
// to configure Parquet reader appropriately
- hadoopConf =
HoodieDataSourceHelper.getConfigurationWithInternalSchema(new
Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath,
validCommits)
+ hadoopConf = embedInternalSchema(new Configuration(conf),
requiredSchema.internalSchema)
Review Comment:
They are
##########
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)) {
+ val nonNullUnionFieldTypes = st.map(f => toAvroType(f.dataType,
nullable = false, f.name, childNameSpace))
+ val unionFieldTypes = if (nullable) {
+ nullSchema +: nonNullUnionFieldTypes
+ } else {
+ nonNullUnionFieldTypes
+ }
+ Schema.createUnion(unionFieldTypes:_*)
+ } else {
+ 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()
+ }
+ fieldsAssembler.endRecord()
}
- fieldsAssembler.endRecord()
// This should never happen.
case other => throw new IncompatibleSchemaException(s"Unexpected type
$other.")
}
- if (nullable && catalystType != NullType) {
+
+ if (nullable && catalystType != NullType && schema.getType !=
Schema.Type.UNION) {
Schema.createUnion(schema, nullSchema)
} else {
schema
}
}
+
+ private def canBeUnion(st: StructType): Boolean = {
Review Comment:
From `AvroDeserializer`
##########
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:
This is actually a very good catch -- `gteqSpark3_1` only matches 3.1 branch
but doesn't for ex 3.2
Its name is misleading. Will update it.
##########
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:
Only modifications are in `apply0` method. I'd recommend comparing it in
this impl against the original one
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala:
##########
@@ -59,10 +59,8 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
// For more details please check HUDI-4161
// NOTE: This override has to mirror semantic of whenever this Relation is
converted into [[HadoopFsRelation]],
// which is currently done for all cases, except when Schema Evolution
is enabled
- override protected val shouldExtractPartitionValuesFromPartitionPath:
Boolean = {
- val enableSchemaOnRead = !internalSchema.isEmptySchema
- !enableSchemaOnRead
- }
+ override protected val shouldExtractPartitionValuesFromPartitionPath:
Boolean =
+ internalSchemaOpt.isEmpty
Review Comment:
It is
--
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]