singhpk234 commented on code in PR #5094:
URL: https://github.com/apache/iceberg/pull/5094#discussion_r902204604


##########
spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.IsNotNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.LeftAnti
+import org.apache.spark.sql.catalyst.plans.LeftOuter
+import org.apache.spark.sql.catalyst.plans.RightOuter
+import org.apache.spark.sql.catalyst.plans.logical.AppendData
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.HintInfo
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.MergeRows
+import org.apache.spark.sql.catalyst.plans.logical.NO_BROADCAST_HASH
+import org.apache.spark.sql.catalyst.plans.logical.NoStatsUnaryNode
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceIcebergData
+import org.apache.spark.sql.catalyst.plans.logical.UpdateAction
+import org.apache.spark.sql.catalyst.plans.logical.WriteDelta
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.expressions.FieldReference
+import org.apache.spark.sql.connector.expressions.NamedReference
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle 
MERGE statements.
+ *
+ * This rule assumes the commands have been fully resolved and all assignments 
have been aligned.
+ * That's why it must be run after AlignRowLevelCommandAssignments.
+ */
+object RewriteMergeIntoTable extends RewriteRowLevelDeltaCommand with 
RewriteRowLevelCommand {
+
+  private final val ROW_FROM_SOURCE = "__row_from_source"
+  private final val ROW_FROM_TARGET = "__row_from_target"
+  private final val ROW_ID = "__row_id"
+
+  private final val ROW_FROM_SOURCE_REF = FieldReference(ROW_FROM_SOURCE)
+  private final val ROW_FROM_TARGET_REF = FieldReference(ROW_FROM_TARGET)
+  private final val ROW_ID_REF = FieldReference(ROW_ID)
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, 
notMatchedActions, None)
+        if m.resolved && m.aligned && matchedActions.isEmpty && 
notMatchedActions.size == 1 =>
+
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r: DataSourceV2Relation =>
+          // NOT MATCHED conditions may only refer to columns in source so 
they can be pushed down
+          val insertAction = notMatchedActions.head.asInstanceOf[InsertAction]
+          val filteredSource = insertAction.condition match {
+            case Some(insertCond) => Filter(insertCond, source)
+            case None => source
+          }
+
+          // when there are no MATCHED actions, use a left anti join to remove 
any matching rows
+          // and switch to using a regular append instead of a row-level merge
+          // only unmatched source rows that match the condition are appended 
to the table
+          val joinPlan = Join(filteredSource, r, LeftAnti, Some(cond), 
JoinHint.NONE)
+
+          val outputExprs = insertAction.assignments.map(_.value)
+          val outputColNames = r.output.map(_.name)
+          val outputCols = outputExprs.zip(outputColNames).map { case (expr, 
name) =>
+            Alias(expr, name)()
+          }
+          val project = Project(outputCols, joinPlan)
+
+          AppendData.byPosition(r, project)
+
+        case p =>
+          throw new AnalysisException(s"$p is not an Iceberg table")
+      }
+
+    case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, 
notMatchedActions, None)
+        if m.resolved && m.aligned && matchedActions.isEmpty =>
+
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r: DataSourceV2Relation =>
+          // when there are no MATCHED actions, use a left anti join to remove 
any matching rows
+          // and switch to using a regular append instead of a row-level merge
+          // only unmatched source rows that match action conditions are 
appended to the table
+          val joinPlan = Join(source, r, LeftAnti, Some(cond), JoinHint.NONE)
+
+          val notMatchedConditions = notMatchedActions.map(actionCondition)
+          val notMatchedOutputs = notMatchedActions.map(actionOutput(_, Nil))
+
+          // merge rows as there are multiple not matched actions
+          val mergeRows = MergeRows(
+            isSourceRowPresent = TrueLiteral,
+            isTargetRowPresent = FalseLiteral,
+            matchedConditions = Nil,
+            matchedOutputs = Nil,
+            notMatchedConditions = notMatchedConditions,
+            notMatchedOutputs = notMatchedOutputs,
+            targetOutput = Nil,
+            rowIdAttrs = Nil,
+            performCardinalityCheck = false,
+            emitNotMatchedTargetRows = false,
+            output = buildMergeRowsOutput(Nil, notMatchedOutputs, r.output),
+            joinPlan)
+
+          AppendData.byPosition(r, mergeRows)
+
+        case p =>
+          throw new AnalysisException(s"$p is not an Iceberg table")
+      }
+
+    case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, 
notMatchedActions, None)
+        if m.resolved && m.aligned =>
+
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, 
_, _) =>
+          val table = buildOperationTable(tbl, MERGE, 
CaseInsensitiveStringMap.empty())

Review Comment:
   have passed empty options based on this discussion in supporting delete pr 
in spark : https://github.com/apache/spark/pull/35395#discussion_r826611125
   
   >These are options passed into newRowLevelOperationBuilder and I thought 
they should come from the SQL operation. For example, if Spark adds a clause 
OPTIONS to its SQL for DELETE, UPDATE, MERGE, then these values will be 
propagated here.
   
   based on the rationale above by @aokolnychyi  I decided to keep them same 
here as well.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to