yihua commented on code in PR #8714:
URL: https://github.com/apache/hudi/pull/8714#discussion_r1198414998


##########
.github/workflows/bot.yml:
##########
@@ -63,6 +63,10 @@ jobs:
             sparkProfile: "spark3.1"
             sparkModules: "hudi-spark-datasource/hudi-spark3.1.x"
 
+          - scalaProfile: "scala-2.12"

Review Comment:
   Would be good to add bundle validation on Spark 3.0.x too in 
`validate-bundles` section



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala:
##########
@@ -114,58 +114,65 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase 
with ScalaAssertionSuppo
     })
   }
 
-  test("Test MergeInto with more than once update actions") {
-    withRecordType()(withTempDir {tmp =>
-      val targetTable = generateTableName
-      spark.sql(
-        s"""
-           |create table ${targetTable} (
-           |  id int,
-           |  name string,
-           |  data int,
-           |  country string,
-           |  ts bigint
-           |) using hudi
-           |tblproperties (
-           |  type = 'cow',
-           |  primaryKey = 'id',
-           |  preCombineField = 'ts'
-           | )
-           |partitioned by (country)
-           |location '${tmp.getCanonicalPath}/$targetTable'
-           |""".stripMargin)
-      spark.sql(
-        s"""
-           |merge into ${targetTable} as target
-           |using (
-           |select 1 as id, 'lb' as name, 6 as data, 'shu' as country, 
1646643193 as ts
-           |) source
-           |on source.id = target.id
-           |when matched then
-           |update set *
-           |when not matched then
-           |insert *
-           |""".stripMargin)
-      spark.sql(
-        s"""
-           |merge into ${targetTable} as target
-           |using (
-           |select 1 as id, 'lb' as name, 5 as data, 'shu' as country, 
1646643196 as ts
-           |) source
-           |on source.id = target.id
-           |when matched and source.data > target.data then
-           |update set target.data = source.data, target.ts = source.ts
-           |when matched and source.data = 5 then
-           |update set target.data = source.data, target.ts = source.ts
-           |when not matched then
-           |insert *
-           |""".stripMargin)
-
-      checkAnswer(s"select id, name, data, country, ts from $targetTable")(
-        Seq(1, "lb", 5, "shu", 1646643196L)
-      )
+  /**
+   * For spark3.0.x didn't support 'UPDATE and DELETE can appear at most once 
in MATCHED clauses in a MERGE statement'
+   * details: 
org.apache.spark.sql.catalyst.parser.AstBuilder#visitMergeIntoTable

Review Comment:
   ```suggestion
      * In Spark 3.0.x, UPDATE and DELETE can appear at most once in MATCHED 
clauses in a MERGE INTO statement.
      * Refer to: 
`org.apache.spark.sql.catalyst.parser.AstBuilder#visitMergeIntoTable`
   ```



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/avro/TestAvroSerDe.scala:
##########
@@ -20,11 +20,12 @@ package org.apache.spark.sql.avro
 import org.apache.avro.generic.GenericData
 import org.apache.hudi.SparkAdapterSupport
 import org.apache.hudi.avro.model.{HoodieMetadataColumnStats, IntWrapper}
+import org.apache.spark.internal.Logging
 import org.apache.spark.sql.avro.SchemaConverters.SchemaType
 import org.junit.jupiter.api.Assertions.assertEquals
 import org.junit.jupiter.api.Test
 
-class TestAvroSerDe extends SparkAdapterSupport {
+class TestAvroSerDe extends SparkAdapterSupport with Logging {

Review Comment:
   Is this necessary for testing?



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystExpressionUtils.scala:
##########
@@ -17,16 +17,9 @@
 
 package org.apache.spark.sql
 
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, 
Expression, Predicate, PredicateHelper}
-import org.apache.spark.sql.execution.datasources.DataSourceStrategy
-
-trait HoodieSpark3CatalystExpressionUtils extends HoodieCatalystExpressionUtils
-  with PredicateHelper {
-
-  override def normalizeExprs(exprs: Seq[Expression], attributes: 
Seq[Attribute]): Seq[Expression] =
-    DataSourceStrategy.normalizeExprs(exprs, attributes)
-
-  override def extractPredicatesWithinOutputSet(condition: Expression,
-                                                outputSet: AttributeSet): 
Option[Expression] =
-    super[PredicateHelper].extractPredicatesWithinOutputSet(condition, 
outputSet)
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, 
Expression}
+abstract class HoodieSpark3CatalystExpressionUtils extends 
HoodieCatalystExpressionUtils {

Review Comment:
   could we keep `with PredicateHelper`?



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieVectorizedParquetRecordReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.parquet;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
+import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
+import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+
+import java.io.IOException;
+import java.time.ZoneId;
+import java.util.HashMap;
+import java.util.Map;
+
+public class Spark30HoodieVectorizedParquetRecordReader extends 
VectorizedParquetRecordReader {

Review Comment:
   I assume these cases are mostly copied from Hudi Spark 3.1.x module.



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/Spark30ResolveHudiAlterTableCommand.scala:
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.common.config.HoodieCommonConfig
+import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.TableChange._
+import org.apache.spark.sql.connector.catalog.TableChange
+import org.apache.spark.sql.hudi.command.Spark30AlterTableCommand
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, NullType, 
StructType}
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+
+import java.util.Locale
+import scala.collection.mutable
+
+/**
+  * Rule to mostly resolve, normalize and rewrite column names based on case 
sensitivity
+  * for alter table column commands.
+  * TODO: we should remove this file when we support datasourceV2 for hoodie 
on spark3.0x

Review Comment:
   ```suggestion
     * TODO: we should remove this file when we support datasourceV2 for hoodie 
on spark3.0.x
   ```



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala:
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.avro
+
+import org.apache.avro.Conversions.DecimalConversion
+import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
+import org.apache.avro.Schema.Type._
+import org.apache.avro.generic._
+import org.apache.avro.util.Utf8
+import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder}
+import org.apache.spark.sql.avro.AvroDeserializer.{createDateRebaseFuncInRead, 
createTimestampRebaseFuncInRead}
+import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, 
UnsafeArrayData}
+import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY
+import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.DataSourceUtils
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+import java.math.BigDecimal
+import java.nio.ByteBuffer
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+/**
+ * A deserializer to deserialize data in avro format to data in catalyst 
format.
+ *
+ * NOTE: This code is borrowed from Spark 3.1.2
+ *       This code is borrowed, so that we can better control compatibility 
w/in Spark minor
+ *       branches (3.2.x, 3.1.x, etc)
+ *
+ *       PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY 
NECESSARY
+ */
+private[sql] class AvroDeserializer(rootAvroType: Schema,

Review Comment:
   Should the code of the class from Spark 3.0.x be copied to here, instead of 
using Spark 3.1.2 version?



##########
pom.xml:
##########
@@ -2238,6 +2239,44 @@
       </activation>
     </profile>
 
+    <profile>
+      <id>spark3.0</id>
+      <properties>
+        <spark3.version>${spark30.version}</spark3.version>
+        <spark.version>${spark3.version}</spark.version>
+        <sparkbundle.version>3.0</sparkbundle.version>
+        <scala.version>${scala12.version}</scala.version>
+        <scala.binary.version>2.12</scala.binary.version>
+        <hudi.spark.module>hudi-spark3.0.x</hudi.spark.module>
+        
<hudi.spark.common.modules.1>hudi-spark3-common</hudi.spark.common.modules.1>
+        <kafka.version>${kafka.spark3.version}</kafka.version>

Review Comment:
   Do we need 
`<scalatest.version>${scalatest.spark3.version}</scalatest.version>`?



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.adapter
+
+import org.apache.avro.Schema
+import org.apache.hudi.Spark30HoodieFileScanRDD
+import org.apache.spark.sql.avro.{HoodieAvroDeserializer, 
HoodieAvroSerializer, HoodieSpark3_0AvroDeserializer, 
HoodieSpark3_0AvroSerializer}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
Expression}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, 
Spark30HoodieParquetFileFormat}
+import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, 
PartitionedFile}
+import org.apache.spark.sql.hudi.SparkAdapter
+import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, 
HoodieSpark3_0ExtendedSqlParser}
+import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, 
StructType}
+import org.apache.spark.sql.vectorized.ColumnarUtils
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.StorageLevel._
+
+/**
+ * Implementation of [[SparkAdapter]] for Spark 3.0.x
+ */
+class Spark3_0Adapter extends BaseSpark3Adapter {
+
+  override def resolveHoodieTable(plan: LogicalPlan): Option[CatalogTable] = {
+    super.resolveHoodieTable(plan).orElse {
+      EliminateSubqueryAliases(plan) match {
+        // First, we need to weed out unresolved plans
+        case plan if !plan.resolved => None
+        // NOTE: When resolving Hudi table we allow [[Filter]]s and 
[[Project]]s be applied
+        //       on top of it
+        case PhysicalOperation(_, _, DataSourceV2Relation(table: CatalogTable, 
_, _, _, _)) if isHoodieTable(table) =>
+          Some(table)
+        case _ => None

Review Comment:
   Is there a way to achieve fallback from V2 to V1 in Spark 3.0?  Without the 
fallback, the performance is affected.



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark30AlterTableCommand.scala:
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.command
+
+import java.net.URI
+import java.nio.charset.StandardCharsets
+import java.util
+import java.util.concurrent.atomic.AtomicInteger
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.DataSourceWriteOptions._
+import org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
+import org.apache.hudi.{AvroConversionUtils, DataSourceOptionsHelper, 
DataSourceUtils}
+import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, 
HoodieInstant}
+import org.apache.hudi.common.table.timeline.HoodieInstant.State
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.util.{CommitUtils, Option}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.internal.schema.InternalSchema
+import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
+import org.apache.hudi.internal.schema.action.TableChanges
+import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
+import org.apache.hudi.internal.schema.utils.{SchemaChangeUtils, SerDeHelper}
+import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager
+import org.apache.hudi.table.HoodieSparkTable
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
+import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange}
+import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, 
DeleteColumn, RemoveProperty, SetProperty}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+// TODO: we should remove this file when we support datasourceV2 for hoodie on 
spark3.0x

Review Comment:
   ```suggestion
   // TODO: we should remove this file when we support datasourceV2 for hoodie 
on spark3.0.x
   ```



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalogUtils.scala:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.connector.expressions.{BucketTransform, 
NamedReference, Transform}
+
+object HoodieSpark30CatalogUtils extends HoodieSpark3CatalogUtils {
+
+  override def unapplyBucketTransform(t: Transform): Option[(Int, 
Seq[NamedReference], Seq[NamedReference])] =
+    t match {
+      case BucketTransform(numBuckets, ref) => Some(numBuckets, Seq(ref), 
Seq.empty)
+      case _ => None
+    }
+

Review Comment:
   nit: remove new line



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/Spark30NestedSchemaPruning.scala:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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, SparkAdapterSupport}
+import org.apache.spark.sql.HoodieSpark3CatalystPlanUtils
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, 
AttributeReference, AttributeSet, Expression, NamedExpression, 
ProjectionOverSchema}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, 
Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, 
StructType}
+
+/**
+ * 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 Spark30NestedSchemaPruning extends Rule[LogicalPlan] {
+  import org.apache.spark.sql.catalyst.expressions.SchemaPruning._
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    if (SQLConf.get.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 planUtils = 
SparkAdapterSupport.sparkAdapter.getCatalystPlanUtils.asInstanceOf[HoodieSpark3CatalystPlanUtils]
+
+        val prunedRelation = outputRelationBuilder(prunedDataSchema)
+        val projectionOverSchema = 
planUtils.projectOverSchema(prunedDataSchema, AttributeSet(output))
+
+        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
+    }
+  }
+
+  private def restoreOriginalOutputNames(
+                                  projectList: Seq[NamedExpression],
+                                  originalNames: Seq[String]): 
Seq[NamedExpression] = {
+    projectList.zip(originalNames).map {
+      case (attr: Attribute, name) => attr.withName(name)
+      case (alias: Alias, name) => if (name == alias.name) {
+        alias
+      } else {
+        AttributeReference(name, alias.dataType, alias.nullable, 
alias.metadata)(alias.exprId, alias.qualifier)
+      }
+      case (other, _) => other
+    }
+  }
+
+

Review Comment:
   ```suggestion
   // NOTE: `pruneDataSchema` and `sortLeftFieldsByRight` functions are copied 
from Spark 3.1.2,
   // as these functions in `SchemaPruning` have bugs in Spark 3.0.2 (see 
SPARK-35096,
   // 
https://github.com/apache/spark/commit/2bbe0a4151f2af00f1105489d5757be28ff278d6)
   ```



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystExpressionUtils.scala:
##########
@@ -17,16 +17,9 @@
 
 package org.apache.spark.sql
 
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, 
Expression, Predicate, PredicateHelper}
-import org.apache.spark.sql.execution.datasources.DataSourceStrategy
-
-trait HoodieSpark3CatalystExpressionUtils extends HoodieCatalystExpressionUtils
-  with PredicateHelper {
-
-  override def normalizeExprs(exprs: Seq[Expression], attributes: 
Seq[Attribute]): Seq[Expression] =
-    DataSourceStrategy.normalizeExprs(exprs, attributes)
-
-  override def extractPredicatesWithinOutputSet(condition: Expression,
-                                                outputSet: AttributeSet): 
Option[Expression] =
-    super[PredicateHelper].extractPredicatesWithinOutputSet(condition, 
outputSet)
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, 
Expression}
+abstract class HoodieSpark3CatalystExpressionUtils extends 
HoodieCatalystExpressionUtils {
+  def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): 
Seq[Expression]
+  def extractPredicatesWithinOutputSet(condition: Expression,
+                                                outputSet: AttributeSet): 
Option[Expression]

Review Comment:
   Scala docs?



-- 
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]

Reply via email to