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


##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala:
##########
@@ -128,9 +131,135 @@ case class HoodieSpark32PlusResolveReferences(spark: 
SparkSession) extends Rule[
           catalogTable.location.toString))
         LogicalRelation(relation, catalogTable)
       }
+    case mO@MatchMergeIntoTable(targetTableO, sourceTableO, _)
+      //// Hudi change: don't want to go to the spark mit resolution so we 
resolve the source and target if they haven't been
+      //
+      if !mO.resolved =>
+      lazy val analyzer = spark.sessionState.analyzer
+      val targetTable = if (targetTableO.resolved) targetTableO else 
analyzer.execute(targetTableO)
+      val sourceTable = if (sourceTableO.resolved) sourceTableO else 
analyzer.execute(sourceTableO)
+      val m = mO.asInstanceOf[MergeIntoTable].copy(targetTable = targetTable, 
sourceTable = sourceTable)
+      //
+      ////

Review Comment:
   docs to add?  Could you check all places where the comment is empty?



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala:
##########
@@ -128,9 +131,135 @@ case class HoodieSpark32PlusResolveReferences(spark: 
SparkSession) extends Rule[
           catalogTable.location.toString))
         LogicalRelation(relation, catalogTable)
       }
+    case mO@MatchMergeIntoTable(targetTableO, sourceTableO, _)
+      //// Hudi change: don't want to go to the spark mit resolution so we 
resolve the source and target if they haven't been
+      //
+      if !mO.resolved =>
+      lazy val analyzer = spark.sessionState.analyzer
+      val targetTable = if (targetTableO.resolved) targetTableO else 
analyzer.execute(targetTableO)
+      val sourceTable = if (sourceTableO.resolved) sourceTableO else 
analyzer.execute(sourceTableO)
+      val m = mO.asInstanceOf[MergeIntoTable].copy(targetTable = targetTable, 
sourceTable = sourceTable)
+      //
+      ////
+      EliminateSubqueryAliases(targetTable) match {
+        case r: NamedRelation if r.skipSchemaResolution =>
+          // Do not resolve the expression if the target table accepts any 
schema.
+          // This allows data sources to customize their own resolution logic 
using
+          // custom resolution rules.
+          m
+
+        case _ =>
+          val newMatchedActions = m.matchedActions.map {
+            case DeleteAction(deleteCondition) =>
+              val resolvedDeleteCondition = deleteCondition.map(
+                resolveExpressionByPlanChildren(_, m))
+              DeleteAction(resolvedDeleteCondition)
+            case UpdateAction(updateCondition, assignments) =>
+              val resolvedUpdateCondition = updateCondition.map(
+                resolveExpressionByPlanChildren(_, m))
+              UpdateAction(
+                resolvedUpdateCondition,
+                // The update value can access columns from both target and 
source tables.
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= false))
+            case UpdateStarAction(updateCondition) =>
+              ////Hudi change: filter out meta fields
+              //
+              val assignments = targetTable.output.filter(a => 
!isMetaField(a.name)).map { attr =>
+                Assignment(attr, UnresolvedAttribute(Seq(attr.name)))
+              }
+              //
+              ////
+              UpdateAction(
+                updateCondition.map(resolveExpressionByPlanChildren(_, m)),
+                // For UPDATE *, the value must from source table.
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= true))
+            case o => o
+          }
+          val newNotMatchedActions = m.notMatchedActions.map {
+            case InsertAction(insertCondition, assignments) =>
+              // The insert action is used when not matched, so its condition 
and value can only
+              // access columns from the source table.
+              val resolvedInsertCondition = insertCondition.map(
+                resolveExpressionByPlanChildren(_, Project(Nil, 
m.sourceTable)))
+              InsertAction(
+                resolvedInsertCondition,
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= true))
+            case InsertStarAction(insertCondition) =>
+              // The insert action is used when not matched, so its condition 
and value can only
+              // access columns from the source table.
+              val resolvedInsertCondition = insertCondition.map(
+                resolveExpressionByPlanChildren(_, Project(Nil, 
m.sourceTable)))
+              ////Hudi change: filter out meta fields
+              //
+              val assignments = targetTable.output.filter(a => 
!isMetaField(a.name)).map { attr =>
+                Assignment(attr, UnresolvedAttribute(Seq(attr.name)))
+              }
+              //
+              ////
+              InsertAction(
+                resolvedInsertCondition,
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= true))
+            case o => o

Review Comment:
   Most of the code here are similar to Spark's `ResolveReferences`.  Could you 
make a note of this and add docs before `case 
mO@MatchMergeIntoTable(targetTableO, sourceTableO, _)` to summarize the custom 
changes?



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala:
##########
@@ -128,9 +131,135 @@ case class HoodieSpark32PlusResolveReferences(spark: 
SparkSession) extends Rule[
           catalogTable.location.toString))
         LogicalRelation(relation, catalogTable)
       }
+    case mO@MatchMergeIntoTable(targetTableO, sourceTableO, _)
+      //// Hudi change: don't want to go to the spark mit resolution so we 
resolve the source and target if they haven't been
+      //
+      if !mO.resolved =>
+      lazy val analyzer = spark.sessionState.analyzer
+      val targetTable = if (targetTableO.resolved) targetTableO else 
analyzer.execute(targetTableO)
+      val sourceTable = if (sourceTableO.resolved) sourceTableO else 
analyzer.execute(sourceTableO)
+      val m = mO.asInstanceOf[MergeIntoTable].copy(targetTable = targetTable, 
sourceTable = sourceTable)
+      //
+      ////
+      EliminateSubqueryAliases(targetTable) match {
+        case r: NamedRelation if r.skipSchemaResolution =>
+          // Do not resolve the expression if the target table accepts any 
schema.
+          // This allows data sources to customize their own resolution logic 
using
+          // custom resolution rules.
+          m
+
+        case _ =>
+          val newMatchedActions = m.matchedActions.map {
+            case DeleteAction(deleteCondition) =>
+              val resolvedDeleteCondition = deleteCondition.map(
+                resolveExpressionByPlanChildren(_, m))
+              DeleteAction(resolvedDeleteCondition)
+            case UpdateAction(updateCondition, assignments) =>
+              val resolvedUpdateCondition = updateCondition.map(
+                resolveExpressionByPlanChildren(_, m))
+              UpdateAction(
+                resolvedUpdateCondition,
+                // The update value can access columns from both target and 
source tables.
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= false))
+            case UpdateStarAction(updateCondition) =>
+              ////Hudi change: filter out meta fields
+              //
+              val assignments = targetTable.output.filter(a => 
!isMetaField(a.name)).map { attr =>
+                Assignment(attr, UnresolvedAttribute(Seq(attr.name)))
+              }
+              //
+              ////
+              UpdateAction(
+                updateCondition.map(resolveExpressionByPlanChildren(_, m)),
+                // For UPDATE *, the value must from source table.
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= true))
+            case o => o
+          }
+          val newNotMatchedActions = m.notMatchedActions.map {
+            case InsertAction(insertCondition, assignments) =>
+              // The insert action is used when not matched, so its condition 
and value can only
+              // access columns from the source table.
+              val resolvedInsertCondition = insertCondition.map(
+                resolveExpressionByPlanChildren(_, Project(Nil, 
m.sourceTable)))
+              InsertAction(
+                resolvedInsertCondition,
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= true))
+            case InsertStarAction(insertCondition) =>
+              // The insert action is used when not matched, so its condition 
and value can only
+              // access columns from the source table.
+              val resolvedInsertCondition = insertCondition.map(
+                resolveExpressionByPlanChildren(_, Project(Nil, 
m.sourceTable)))
+              ////Hudi change: filter out meta fields
+              //
+              val assignments = targetTable.output.filter(a => 
!isMetaField(a.name)).map { attr =>
+                Assignment(attr, UnresolvedAttribute(Seq(attr.name)))
+              }
+              //
+              ////
+              InsertAction(
+                resolvedInsertCondition,
+                resolveAssignments(assignments, m, resolveValuesWithSourceOnly 
= true))
+            case o => o

Review Comment:
   Also, is the code copied from Spark 3.2?  Any difference among Spark 3.2, 
3.3, and 3.4?



##########
hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/catalyst/analysis/HoodieSpark30Analysis.scala:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.catalyst.analysis
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, 
ResolveLambdaVariables, UnresolvedAttribute, UnresolvedExtractValue, 
caseInsensitiveResolution, withPosition}
+import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentDate, 
CurrentTimestamp, Expression, ExtractValue, GetStructField, LambdaFunction}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.toPrettySQL
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
+
+/**
+ * NOTE: Taken from HoodieSpark2Analysis and modified to resolve source and 
target tables if not already resolved
+ *
+ *       PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY 
NECESSARY
+ */
+object HoodieSpark30Analysis {

Review Comment:
   For Spark 3.0 and 3.1, have you checked if the code here is different from 
Spark's `ResolveReferences`.  Given we introduce the custom rule here, we 
should still match the implementation of `ResolveReferences` in Spark except 
for the custom logic you added.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to