You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/26 13:13:11 UTC

[GitHub] [iceberg] aokolnychyi opened a new pull request #1986: Spark: Add a rule to align updates in MERGE operations

aokolnychyi opened a new pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986


   This PR adds a rule that aligns assignments in MERGE operations with target table columns.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#issuecomment-752637602


   This PR is ready for another review round while I am working on more tests.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552447202



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct

Review comment:
       Yeah, correct.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549826688



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))

Review comment:
       Yeah, you are right. I can simplify this a bit further.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551841131



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column

Review comment:
       We do that check only for INSERT actions. So you are right we can only hit case 2 there cause we know INSERT can only have top-level columns and we validate they are unique upfront. However, this logic is also applied to UPDATE actions where we don't do that check.
   
   UPDATEs are a bit more tricky as they can have references to nested columns. We could try to do the uniqueness check early but we should be careful. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549540194



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))
+      if (assignment.isEmpty) {
+        throw new AnalysisException(
+          s"Cannot find column '${targetAttr.name}' of the target table among " +
+          s"the INSERT columns: ${insertCols.mkString(", ")}. " +
+          "INSERT clauses must provide values for all columns of the target table.")
+      }
+
+      val key = assignment.get.key
+      val value = assignment.get.value
+      Assignment(key, castIfNeeded(value, key.dataType, resolver))

Review comment:
       Why not cast directly to the output column's data type?
   
   Seems like this should cast to the types from `MergeIntoTable.output` so that all of the assignments produce the same type. Are all of the assignment keys for a column guaranteed to have the same output type?
   
   Also, if column resolution adds a projection with a cast to the a table output type, can these expressions be optimized to remove the duplicate cast? If not, we may want to cast directly to the table's type and update the output of the `MergeIntoTable` node to be identical to the table so column resolution is done here rather than in the normal write resolution rule.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552933022



##########
File path: spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java
##########
@@ -132,4 +134,40 @@ protected void assertEquals(String context, List<Object[]> expectedRows, List<Ob
   protected static String dbPath(String dbName) {
     return metastore.getDatabasePath(dbName);
   }
+
+  protected void withSQLConf(Map<String, String> conf, Action action) {
+    SQLConf sqlConf = SQLConf.get();
+
+    Map<String, String> currentConfValues = Maps.newHashMap();
+    conf.keySet().forEach(confKey -> {
+      if (sqlConf.contains(confKey)) {
+        String currentConfValue = sqlConf.getConfString(confKey);
+        currentConfValues.put(confKey, currentConfValue);
+      }
+    });
+
+    conf.forEach((confKey, confValue) -> {
+      if (SQLConf.staticConfKeys().contains(confKey)) {
+        throw new RuntimeException("Cannot modify the value of a static config: " + confKey);
+      }
+      sqlConf.setConfString(confKey, confValue);
+    });
+
+    try {
+      action.invoke();
+    } finally {
+      conf.forEach((confKey, confValue) -> {
+        if (currentConfValues.containsKey(confKey)) {
+          sqlConf.setConfString(confKey, currentConfValues.get(confKey));
+        } else {
+          sqlConf.unsetConf(confKey);
+        }
+      });
+    }
+  }
+
+  @FunctionalInterface
+  protected interface Action {
+    void invoke();
+  }

Review comment:
       Nit: this looks like `Runnable` with different names. If we are just introducing it to pass a lambda, we could probably use `Runnable` instead.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549782255



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")
+        case (from, to) if requiresCast(from, to) => cast(expr, dataType)
+        case _ => expr
+      }
+  }
+
+  private def requiresCast(from: DataType, to: DataType): Boolean = {
+    !DataType.equalsIgnoreCaseAndNullability(from, to)
+  }
+
+  private def isValidCast(from: StructType, to: StructType, resolver: Resolver): Boolean = {
+    from.length == to.length && from.exists { f => to.exists(t => resolver(f.name, t.name))}
+  }
+
+  protected def getNameParts(expr: Expression): Seq[String] = expr match {

Review comment:
       Or `asAssignReference`?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551628799



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d

Review comment:
       The equivalent `match` expression for `notMatchedActions` has a default case that throws "Not matched actions can only contain INSERT". Should there be a similar case here to reject any INSERT action?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550279680



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))
+      if (assignment.isEmpty) {
+        throw new AnalysisException(
+          s"Cannot find column '${targetAttr.name}' of the target table among " +
+          s"the INSERT columns: ${insertCols.mkString(", ")}. " +
+          "INSERT clauses must provide values for all columns of the target table.")
+      }
+
+      val key = assignment.get.key
+      val value = assignment.get.value
+      Assignment(key, castIfNeeded(value, key.dataType, resolver))

Review comment:
       I am not sure I fully understood. Could you check whether the current implementation is similar to what you mean?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550479955



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(

Review comment:
       Done.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549635981



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))

Review comment:
       I think a simple zip won't work since the assignments haven't been aligned yet. Plus, we have to use the resolver for equality so cannot have O(1) lookups.
   
   That said, we can avoid the main problem of constantly calling `getNameParts`. Let me do that.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549786142



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")
+        case (from, to) if requiresCast(from, to) => cast(expr, dataType)

Review comment:
       This inserts a cast regardless of whether it is safe to cast. There is also now an assignment policy that changes which casts should be inserted. I think this should apply the same logic as the table output resolver, just using the assignment names. What do you think of the logic in [`TableOutputResolver.checkField`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala#L84-L130)?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550480118



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    targetTable.output.map { targetAttr =>
+      val assignment = assignmentMap
+        .find { case (name, _) => resolver(name, targetAttr.name) }
+        .map { case (_, assignment) => assignment }
+
+      if (assignment.isEmpty) {
+        throw new AnalysisException(

Review comment:
       I think we should set it to null instead of throwing an exception.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550279484



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))

Review comment:
       Done. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550291799



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)

Review comment:
       > In particular, what should happen if one sets a struct with fields (b, a) to a struct column with fields (a, b)? Should we match the fields by name or should we match them by position?
   
   SQL writes are always by position unless you have `(names...) VALUES (values...)`, but that's not the case for these structs. So I think think the right behavior for SQL is by position.
   
   Because dataframe columns don't have an obvious position, the expectation for users is that the writes happen by name. That's why we added `byName` variants of the logical plans. I think that extends to nested structs as well because nested structs are easy to produce by some conversion from an object. When converting from an object using an `Encoder`, there is no column order guarantee so it is reasonable to assume that columns will be written by name.
   
   Because we don't currently support dataframe merge into, I think that we should move forward with the behavior you've added that matches the insert behavior. We can add the `byName` flag later when we add a dataframe API.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549627424



##########
File path: LICENSE
##########
@@ -289,3 +289,12 @@ Copyright: 2011-2018 The Apache Software Foundation
 Home page: https://spark.apache.org/
 License: https://www.apache.org/licenses/LICENSE-2.0
 
+--------------------------------------------------------------------------------
+
+This product includes code from Delta Lake.
+
+* AssignmentAlignmentSupport is an independent development but was inspired by an equivalent rule in Delta.

Review comment:
       It was used as a reference. I did not simply copy the code as is so I added "inspired by" to give credit to the original authors. The rules should be logically equivalent and should be part of Spark, not data sources.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551853117



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct

Review comment:
       I think the existing logic matches Postgres.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549537883



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))

Review comment:
       Why add `[` and `]`? Those seem distracting to me.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551642052



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields

Review comment:
       I find the logic here hard to follow because `applyUpdates` and `applyStructUpdates` call one another to recursively traverse the schema. It is also strange that the current column is passed as `NamedExpression`, which will be created with an alias for each level.
   
   I think it would be easier to read and a little cleaner if you removed `col` and just traversed the schema in one method. It should be possible to build an expression equivalent to `col` from `namePrefix` and each field name.
   
   I don't think this is a blocker, though.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551643731



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct

Review comment:
       What is the correct behavior here? Reject it?
   
   I think what would currently happen is it would fail if any existing field were accessed, but it would succeed if all fields were assigned. Is that correct?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551637455



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))

Review comment:
       Minor: I prefer not to add `[` and `]` unless they add value or form a literal.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550279837



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")

Review comment:
       Resolving this as the code changed. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550480230



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)

Review comment:
       These TODOs require an actual implementation.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552443382



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct
+  // TODO: tests for subqueries in conditions
+
+  @Test
+  public void testMergeWithNonExistingColumns() {
+    createAndInitNestedColumnsTable();

Review comment:
       I tried to save some extra lines but I agree it is not readable.
   I'll change this to:
   
   ```
   createAndInitTable("id INT, c STRUCT<n1:INT,n2:STRUCT<dn1:INT,dn2:INT>>");
   ```
   
   Once we have tests for real cases, we can offer a method that will accept a JSON body.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550209331



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      tableAttr: NamedExpression,
+      expr: Expression,
+      resolver: Resolver): Expression = {
+
+    val storeAssignmentPolicy = conf.storeAssignmentPolicy
+
+    // run the type check and catch type errors
+    storeAssignmentPolicy match {
+      case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI =>
+        if (expr.nullable && !tableAttr.nullable) {
+          throw new AnalysisException(
+            s"Cannot write nullable values to non-null column '${tableAttr.name}'")
+        }
+
+        val errors = new mutable.ArrayBuffer[String]()
+        val canWrite = DataType.canWrite(
+          expr.dataType, tableAttr.dataType, byName = false, resolver, tableAttr.name,
+          storeAssignmentPolicy, err => errors += err)
+
+        if (!canWrite) {
+          throw new AnalysisException(s"Cannot write incompatible data:\n- ${errors.mkString("\n- ")}")
+        }
+
+      case _ => // OK
+    }
+
+    storeAssignmentPolicy match {
+      case _ if tableAttr.dataType.sameType(expr.dataType) =>
+        expr
+      case StoreAssignmentPolicy.ANSI =>
+        AnsiCast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone))
+      case _ =>
+        Cast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone))
+    }
+  }
+
+  implicit protected def asAssignmentReference(expr: Expression): Seq[String] = expr match {

Review comment:
       I made this implicit.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550209032



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(

Review comment:
       I've reworked `castIfNeeded` completely. Now, it is closer to `checkField` in `TableOutputResolver`.
   
   I should probably add this to the licence.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551642263



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))

Review comment:
       Updates with empty references will be removed by the filter in `applyUpdates`, right?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549629387



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))

Review comment:
       Spark is going to run `AlignMergeIntoTable` only once after the resolution is done and fully successful (we inject this rule in post-hoc resolution phase and check that the plan is fully resolved before applying this logic). 
   
   The purpose of the rule is to align already resolved expressions and do it only once.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551644760



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct
+  // TODO: tests for subqueries in conditions
+
+  @Test
+  public void testMergeWithNonExistingColumns() {
+    createAndInitNestedColumnsTable();
+    createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }");
+
+    AssertHelpers.assertThrows("Should complain about the invalid top-level column",
+        AnalysisException.class, "cannot resolve '`t.invalid_col`'",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.invalid_col = s.c2", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about the invalid nested column",
+        AnalysisException.class, "No such struct field invalid_col",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.invalid_col = s.c2", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about the invalid top-level column",
+        AnalysisException.class, "cannot resolve '`invalid_col`'",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, invalid_col) VALUES (s.c1, null)", tableName);
+        });
+  }
+
+  @Test
+  public void testMergeWithInvalidColumnsInInsert() {
+    createAndInitNestedColumnsTable();
+    createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }");
+
+    AssertHelpers.assertThrows("Should complain about the nested column",
+        AnalysisException.class, "Nested fields are not supported inside INSERT clauses",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, c.n2) VALUES (s.c1, null)", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about duplicate columns",
+        AnalysisException.class, "Duplicate column names inside INSERT clause",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, id) VALUES (s.c1, null)", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about missing columns",
+        AnalysisException.class, "must provide values for all columns of the target table",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +

Review comment:
       I don't think that this should have a `WHEN MATCHED` case, to ensure that it is failing for the `NOT MATCHED THEN INSERT`.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551847484



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields

Review comment:
       I am going to increase the allowed method length to 100. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551862816



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct
+  // TODO: tests for subqueries in conditions
+
+  @Test
+  public void testMergeWithNonExistingColumns() {
+    createAndInitNestedColumnsTable();
+    createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }");
+
+    AssertHelpers.assertThrows("Should complain about the invalid top-level column",
+        AnalysisException.class, "cannot resolve '`t.invalid_col`'",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.invalid_col = s.c2", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about the invalid nested column",
+        AnalysisException.class, "No such struct field invalid_col",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.invalid_col = s.c2", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about the invalid top-level column",
+        AnalysisException.class, "cannot resolve '`invalid_col`'",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, invalid_col) VALUES (s.c1, null)", tableName);
+        });
+  }
+
+  @Test
+  public void testMergeWithInvalidColumnsInInsert() {
+    createAndInitNestedColumnsTable();
+    createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }");
+
+    AssertHelpers.assertThrows("Should complain about the nested column",
+        AnalysisException.class, "Nested fields are not supported inside INSERT clauses",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, c.n2) VALUES (s.c1, null)", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about duplicate columns",
+        AnalysisException.class, "Duplicate column names inside INSERT clause",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, id) VALUES (s.c1, null)", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about missing columns",
+        AnalysisException.class, "must provide values for all columns of the target table",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +

Review comment:
       Done.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549629549



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))

Review comment:
       Let me fix it.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550209583



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for action conditions
+  // TODO: tests for writing nullable to not nullable, incompatible arrays, structs, atomic types

Review comment:
       I am going to address the last two TODOs before this PR is merged.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549639479



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))
+      if (assignment.isEmpty) {
+        throw new AnalysisException(
+          s"Cannot find column '${targetAttr.name}' of the target table among " +
+          s"the INSERT columns: ${insertCols.mkString(", ")}. " +
+          "INSERT clauses must provide values for all columns of the target table.")
+      }
+
+      val key = assignment.get.key
+      val value = assignment.get.value
+      Assignment(key, castIfNeeded(value, key.dataType, resolver))

Review comment:
       I think you are correct we should cast to `targetAttr.dataType`. The resolution should be done before this rule so we should be safe with `castIfNeeded` not adding anything extra if it is not needed.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549796586



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))
+      if (assignment.isEmpty) {
+        throw new AnalysisException(
+          s"Cannot find column '${targetAttr.name}' of the target table among " +
+          s"the INSERT columns: ${insertCols.mkString(", ")}. " +
+          "INSERT clauses must provide values for all columns of the target table.")
+      }
+
+      val key = assignment.get.key
+      val value = assignment.get.value
+      Assignment(key, castIfNeeded(value, key.dataType, resolver))

Review comment:
       Good context that this is a post-hoc rule, but I think that this might introduce problems.
   
   Where does the output of `MergeIntoTable` come from? Table resolution is only going to add projections, so I don't think we can cast directly to the table type, we need to cast to the expected output type. I think that should be the same?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549783773



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")

Review comment:
       I think would be a bit easier to read this code if the `isValidCast` check were in a single case for structs. That way you don't have to compare the case clauses to see what's different. It also avoids running `requiresCast` twice.
   
   ```scala
     if (!isValidCast(from, to resolver)) {
       throw new AnalysisException(...)
     }
     ...
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552122468



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column

Review comment:
       This seems fine to me.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#issuecomment-755177604


   I think this one is ready for a final review round.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549537584



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")
+        case (from, to) if requiresCast(from, to) => cast(expr, dataType)
+        case _ => expr
+      }
+  }
+
+  private def requiresCast(from: DataType, to: DataType): Boolean = {
+    !DataType.equalsIgnoreCaseAndNullability(from, to)
+  }
+
+  private def isValidCast(from: StructType, to: StructType, resolver: Resolver): Boolean = {
+    from.length == to.length && from.exists { f => to.exists(t => resolver(f.name, t.name))}
+  }
+
+  protected def getNameParts(expr: Expression): Seq[String] = expr match {

Review comment:
       What about `getInsertReference` or `asInsertReference`? Seems like this is actually quite specific to the references that we expect in the left side of an `INSERT` clause.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550289354



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(

Review comment:
       Yes, please do.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549631015



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))

Review comment:
       Good point.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550168710



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)

Review comment:
       I gave this a bit of thought.
   
   We should discuss whether to resolve nested struct fields by name or by position. In particular, what should happen if one sets a struct with fields (b, a) to a struct column with fields (a, b)? Should we match the fields by name or should we match them by position?
   
   I think actions in MERGE operations are resolved by position.
   
   For example, `INSERT (c1, c2) VALUES (s.c2, s.c1)` as a branch of MERGE will be resolved as `c1 = s.c2, c2 = s.c1`. I think we can follow what Spark does for regular SQL INSERTs and resolve struct fields by position, not by name like this PR does now. 
   
   Not resolving nested fields by name may lead to unexpected results from the user perspective but we have this problem inside INSERT statements already. Spark ignores column names inside INSERTs and adds aliases if needed.
   
   Are we ok ignoring column names inside MERGE too? I guess that would be reasonable and would match the existing behavior for SQL INSERTs. If so, assigning a struct (b, a) as struct (a, b) may be legal but the new value of `b` will be applied to `a` column.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549831888



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")
+        case (from, to) if requiresCast(from, to) => cast(expr, dataType)
+        case _ => expr
+      }
+  }
+
+  private def requiresCast(from: DataType, to: DataType): Boolean = {
+    !DataType.equalsIgnoreCaseAndNullability(from, to)
+  }
+
+  private def isValidCast(from: StructType, to: StructType, resolver: Resolver): Boolean = {
+    from.length == to.length && from.exists { f => to.exists(t => resolver(f.name, t.name))}
+  }
+
+  protected def getNameParts(expr: Expression): Seq[String] = expr match {

Review comment:
       I'd go for `asAssignmentReference` and making the method implicit.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549640430



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")
+        case (from, to) if requiresCast(from, to) => cast(expr, dataType)
+        case _ => expr
+      }
+  }
+
+  private def requiresCast(from: DataType, to: DataType): Boolean = {
+    !DataType.equalsIgnoreCaseAndNullability(from, to)
+  }
+
+  private def isValidCast(from: StructType, to: StructType, resolver: Resolver): Boolean = {
+    from.length == to.length && from.exists { f => to.exists(t => resolver(f.name, t.name))}
+  }
+
+  protected def getNameParts(expr: Expression): Seq[String] = expr match {

Review comment:
       It is also used for UPDATE SET clause too.
   
   What about `asMultipartIdentifier`? We can make it implicit if we want to.

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)
+        case (from: StructType, to: StructType) if requiresCast(from, to) && isValidCast(from, to, resolver) =>
+          val fieldExprs = to.flatMap { field =>
+            val fieldName = Literal(field.name)
+            val fieldExpr = ExtractValue(expr, fieldName, resolver)
+            Seq(fieldName, castIfNeeded(fieldExpr, field.dataType, resolver))
+          }
+          CreateNamedStruct(fieldExprs)
+        case (from: StructType, to: StructType) if requiresCast(from, to) =>
+          throw new AnalysisException(s"Invalid assignments: cannot cast $from to $to")
+        case (from, to) if requiresCast(from, to) => cast(expr, dataType)
+        case _ => expr
+      }
+  }
+
+  private def requiresCast(from: DataType, to: DataType): Boolean = {
+    !DataType.equalsIgnoreCaseAndNullability(from, to)
+  }
+
+  private def isValidCast(from: StructType, to: StructType, resolver: Resolver): Boolean = {
+    from.length == to.length && from.exists { f => to.exists(t => resolver(f.name, t.name))}
+  }
+
+  protected def getNameParts(expr: Expression): Seq[String] = expr match {

Review comment:
       It is also used for UPDATE SET clauses too.
   
   What about `asMultipartIdentifier`? We can make it implicit if we want to.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550860126



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    targetTable.output.map { targetAttr =>
+      val assignment = assignmentMap
+        .find { case (name, _) => resolver(name, targetAttr.name) }
+        .map { case (_, assignment) => assignment }
+
+      if (assignment.isEmpty) {
+        throw new AnalysisException(

Review comment:
       Well, this would work fine only if the assignment mode isn't `legacy`.
   
   In `strict` and `ansi`, there will be an exception in `castIfNeeded` if the column is not nullable. In `legacy`, this will go through.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549642137



##########
File path: LICENSE
##########
@@ -289,3 +289,12 @@ Copyright: 2011-2018 The Apache Software Foundation
 Home page: https://spark.apache.org/
 License: https://www.apache.org/licenses/LICENSE-2.0
 
+--------------------------------------------------------------------------------
+
+This product includes code from Delta Lake.
+
+* AssignmentAlignmentSupport is an independent development but was inspired by an equivalent rule in Delta.

Review comment:
       Updated.

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))

Review comment:
       Done.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549539207



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))

Review comment:
       This could avoid calling `getNameParts` by creating a map out of `insertCols` and `assignments`:
   
   ```
   val assignmentMap = insertCols.zip(assigments).toMap
   ```
   
   That would be better since this is going to call `getNameParts` for every assignment until a matching one is found.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551629791



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    targetTable.output.map { targetAttr =>
+      val assignment = assignmentMap
+        .find { case (name, _) => resolver(name, targetAttr.name) }
+        .map { case (_, assignment) => assignment }
+
+      if (assignment.isEmpty) {
+        throw new AnalysisException(

Review comment:
       Should this check if the mode is legacy and add null otherwise?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551852783



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct

Review comment:
       I think rejecting will be too restrictive. That would mean we won't be able to update any nullable struct.
   
   Here is what Postgres does:
   
   ```
   postgres=# INSERT INTO t_100 (id, c) VALUES (2, null);
   INSERT 0 1
   
   postgres=# SELECT * FROM t_100;
    id |   c   
   ----+-------
     1 | (1,2)
     2 | 
   (2 rows)
   
   postgres=# UPDATE t_100 SET c.n1 = -1;
   UPDATE 2
   
   postgres=# SELECT * FROM t_100;
    id |   c    
   ----+--------
     1 | (-1,2)
     2 | (-1,)
   (2 rows)
   
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549536305



##########
File path: LICENSE
##########
@@ -289,3 +289,12 @@ Copyright: 2011-2018 The Apache Software Foundation
 Home page: https://spark.apache.org/
 License: https://www.apache.org/licenses/LICENSE-2.0
 
+--------------------------------------------------------------------------------
+
+This product includes code from Delta Lake.
+
+* AssignmentAlignmentSupport is an independent development but was inspired by an equivalent rule in Delta.

Review comment:
       Can you be more specific about "inspired by"? Did you use the code from Delta as a reference? Did you use the same test cases?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#issuecomment-754568284


   I've pushed more changes and I feel like this PR is converging. We still a few open questions but I feel like we can address them in one more review round.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552126645



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct
+  // TODO: tests for subqueries in conditions
+
+  @Test
+  public void testMergeWithNonExistingColumns() {
+    createAndInitNestedColumnsTable();

Review comment:
       I think it would be much easier to read these tests if instead of using these methods, you just embedded the `CREATE TABLE` here, like this:
   
   ```java
     sql("CREATE TABLE %s (..., complex struct<c1 int, c2 int) ...", tableName);
     initTable(tableName);
   ```
   
   That way, it is easy to see what the available columns are without needing to refer back to the base class. I found it easy to see the structure and data for `source`, but kept going back to see what the target table's schema was.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551844975



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields

Review comment:
       I did this as our Scala checks complained about the method length. Let me see if I can revert this change.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549789039



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)

Review comment:
       This means that assignments within a struct are always done by name and never by position? In table output resolution, it looks like structs must match the expected order and if writing by name then the names must match. I would probably use the same logic as in [`canWrite`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala#L493).




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551636799



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column

Review comment:
       If this happened, then two expressions in the left side of the assignment expression would match, right?
   
   ```sql
   INSERT (a.b, a.b) VALUES (2, 3)
   ```
   
   Since we know that the assignment references on the left are unique from an earlier check, we only have the second case, right?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551835953



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    targetTable.output.map { targetAttr =>
+      val assignment = assignmentMap
+        .find { case (name, _) => resolver(name, targetAttr.name) }
+        .map { case (_, assignment) => assignment }
+
+      if (assignment.isEmpty) {
+        throw new AnalysisException(

Review comment:
       I guess that would be reasonable.
   
   Hm, I wanted to match the behavior in INSERT but did not manage to make it work with an explicit column list.
   
   ```
   INSERT INTO t (col1, col2, ...) VALUES (v1, v2, ...)
   ```
   
   Shall we leave it as is for now?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551863154



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d

Review comment:
       I think Spark validates them while parsing but I added for consistency.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551861599



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct
+  // TODO: tests for subqueries in conditions
+
+  @Test
+  public void testMergeWithNonExistingColumns() {
+    createAndInitNestedColumnsTable();
+    createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }");
+
+    AssertHelpers.assertThrows("Should complain about the invalid top-level column",
+        AnalysisException.class, "cannot resolve '`t.invalid_col`'",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.invalid_col = s.c2", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about the invalid nested column",
+        AnalysisException.class, "No such struct field invalid_col",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.invalid_col = s.c2", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about the invalid top-level column",
+        AnalysisException.class, "cannot resolve '`invalid_col`'",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, invalid_col) VALUES (s.c1, null)", tableName);
+        });
+  }
+
+  @Test
+  public void testMergeWithInvalidColumnsInInsert() {
+    createAndInitNestedColumnsTable();
+    createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }");
+
+    AssertHelpers.assertThrows("Should complain about the nested column",
+        AnalysisException.class, "Nested fields are not supported inside INSERT clauses",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, c.n2) VALUES (s.c1, null)", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about duplicate columns",
+        AnalysisException.class, "Duplicate column names inside INSERT clause",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +
+              "WHEN NOT MATCHED THEN " +
+              "  INSERT (id, id) VALUES (s.c1, null)", tableName);
+        });
+
+    AssertHelpers.assertThrows("Should complain about missing columns",
+        AnalysisException.class, "must provide values for all columns of the target table",
+        () -> {
+          sql("MERGE INTO %s t USING source s " +
+              "ON t.id == s.c1 " +
+              "WHEN MATCHED THEN " +
+              "  UPDATE SET t.c.n2.dn1 = s.c2 " +

Review comment:
       +1




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549795441



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))

Review comment:
       I thought insertCols was derived from assignments like `assigments.map(getNameParts(_.key).head)`, so we know that they are equivalent. You're right about the resolver, though.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#issuecomment-751354619


   @rdblue @dilipbiswal, could you review, please?


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552448720



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct
+  // TODO: tests for subqueries in conditions
+
+  @Test
+  public void testMergeWithNonExistingColumns() {
+    createAndInitNestedColumnsTable();

Review comment:
       Updated.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549537791



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)

Review comment:
       Should this be `colName` since it is only one identifier?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550168710



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)

Review comment:
       I gave this a bit of thought.
   
   We should discuss whether to resolve nested struct fields by name or by position. In particular, what should happen if one sets a struct with fields (b, a) to a struct column with fields (a, b)? Should we match the fields by name or should we match them by position?
   
   I think actions in MERGE operations are resolved by position.
   
   For example, `INSERT (c1, c2) VALUES (s.c2, s.c1)` will be resolved as `c1 = s.c2, c2 = s.c1`. I think we can follow what Spark does for INSERTs and resolve struct fields by position, not by name like this PR does now. 
   
   Not resolving nested fields by name may lead to unexpected results from the user perspective but we have this problem inside INSERT statements already. Spark ignores column names inside INSERTs and adds aliases if needed.
   
   Are we ok ignoring column names inside MERGE too? I guess that would be reasonable and would match the existing behavior for SQL INSERTs. If so, assigning a struct (b, a) as struct (a, b) may be legal but the new value of `b` will be applied to `a` column.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550208667



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap

Review comment:
       We build the map a bit earlier now.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549536644



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>

Review comment:
       Nit: I would probably add a newline before this case since the last block was so large.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551852783



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct

Review comment:
       I think rejecting will be too restrictive. That would mean we won't be able to update any nullable structs.
   
   Here is what Postgres does:
   
   ```
   postgres=# INSERT INTO t_100 (id, c) VALUES (2, null);
   INSERT 0 1
   
   postgres=# SELECT * FROM t_100;
    id |   c   
   ----+-------
     1 | (1,2)
     2 | 
   (2 rows)
   
   postgres=# UPDATE t_100 SET c.n1 = -1;
   UPDATE 2
   
   postgres=# SELECT * FROM t_100;
    id |   c    
   ----+--------
     1 | (-1,2)
     2 | (-1,)
   (2 rows)
   
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550426597



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.types.{DataType, NullType, StructField, StructType}
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(nameParts: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(getNameParts(a.key), a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.nameParts.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(update.expr, col.dataType, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.nameParts).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(nameParts = u.nameParts.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.nameParts match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      expr: Expression,
+      dataType: DataType,
+      resolver: Resolver): Expression = expr match {
+    // some types cannot be casted from NullType (e.g. StructType)
+    case Literal(value, NullType) => Literal(value, dataType)
+    case _ =>
+      (expr.dataType, dataType) match {
+        // resolve structs by name if they they have the same number of fields and their names match
+        // e.g., it is ok to set a struct with fields (a, b) as another struct with fields (b, a)
+        // it is invalid to a set a struct with fields (a, d) as another struct with fields (a, b)

Review comment:
       Well, we do have `(names...) VALUES (values...)` in the insert action:
   
   ```
   WHEN NOT MATCHED THEN
     INSERT (c1, c2) VALUES (v1, v2)
   ```
   
   But I am not sure whether that means nested structs must be resolved by name.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551835953



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    targetTable.output.map { targetAttr =>
+      val assignment = assignmentMap
+        .find { case (name, _) => resolver(name, targetAttr.name) }
+        .map { case (_, assignment) => assignment }
+
+      if (assignment.isEmpty) {
+        throw new AnalysisException(

Review comment:
       I guess that would be reasonable.
   
   Hm, I wanted to match the behavior in INSERT but did not manage to make it work with an explicit column list.
   
   ```
   INSERT INTO t (col1, col2, ...) VALUES (v1, v2, ...)
   ```
   
   Shall we leave it as is for now? The current option is the most restrictive.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550209221



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      tableAttr: NamedExpression,
+      expr: Expression,
+      resolver: Resolver): Expression = {
+
+    val storeAssignmentPolicy = conf.storeAssignmentPolicy
+
+    // run the type check and catch type errors
+    storeAssignmentPolicy match {
+      case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI =>
+        if (expr.nullable && !tableAttr.nullable) {
+          throw new AnalysisException(
+            s"Cannot write nullable values to non-null column '${tableAttr.name}'")

Review comment:
       Now we should have proper validation for writing nullable values to non-null columns.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549693852



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected abstract Map<String, String> extraTableProperties();

Review comment:
       This one is redundant, I'll remove it.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549537883



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))

Review comment:
       Why add `[` and `]`? Those seem distracting to me. I'd also add a space after `,` to make it more readable.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551846426



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))

Review comment:
       Yep, when we ensure there is a match by prefix.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549641871



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    // init a map of assignments to avoid calling getNameParts many times
+    val assignmentMap = assignments.map(a => getNameParts(a.key).head -> a).toMap

Review comment:
       Here is what I came up with, @rdblue. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r553056373



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,192 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              // build field expressions
+              val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+                Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+              }
+
+              // recursively apply this method on nested fields
+              val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+              val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+              // construct a new struct with updated field expressions
+              toNamedStruct(fields, updatedFieldExprs)
+
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString(", "))
+      }
+    }
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      tableAttr: NamedExpression,
+      expr: Expression,
+      resolver: Resolver): Expression = {
+
+    val storeAssignmentPolicy = conf.storeAssignmentPolicy
+
+    // run the type check and catch type errors
+    storeAssignmentPolicy match {
+      case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI =>
+        if (expr.nullable && !tableAttr.nullable) {
+          throw new AnalysisException(
+            s"Cannot write nullable values to non-null column '${tableAttr.name}'")
+        }
+
+        // we use byName = true to catch cases when struct field names don't match
+        // e.g. a struct with fields (a, b) is assigned as a struct with fields (a, c) or (b, a)
+        val errors = new mutable.ArrayBuffer[String]()
+        val canWrite = DataType.canWrite(
+          expr.dataType, tableAttr.dataType, byName = true, resolver, tableAttr.name,
+          storeAssignmentPolicy, err => errors += err)
+
+        if (!canWrite) {
+          throw new AnalysisException(s"Cannot write incompatible data:\n- ${errors.mkString("\n- ")}")
+        }
+
+      case _ => // OK
+    }
+
+    storeAssignmentPolicy match {
+      case _ if tableAttr.dataType.sameType(expr.dataType) =>
+        expr
+      case StoreAssignmentPolicy.ANSI =>
+        AnsiCast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone))
+      case _ =>
+        Cast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone))
+    }
+  }
+
+  implicit protected def asAssignmentReference(expr: Expression): Seq[String] = expr match {

Review comment:
       Does this need to be implicit? It is only used in a `map` call and that looks like it would work with a normal function.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549537791



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)

Review comment:
       Should this be `colName` since it is only one identifier?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550291949



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))
+      if (assignment.isEmpty) {
+        throw new AnalysisException(
+          s"Cannot find column '${targetAttr.name}' of the target table among " +
+          s"the INSERT columns: ${insertCols.mkString(", ")}. " +
+          "INSERT clauses must provide values for all columns of the target table.")
+      }
+
+      val key = assignment.get.key
+      val value = assignment.get.value
+      Assignment(key, castIfNeeded(value, key.dataType, resolver))

Review comment:
       Will do.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r550210473



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      tableAttr: NamedExpression,
+      expr: Expression,
+      resolver: Resolver): Expression = {
+
+    val storeAssignmentPolicy = conf.storeAssignmentPolicy
+
+    // run the type check and catch type errors
+    storeAssignmentPolicy match {
+      case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI =>
+        if (expr.nullable && !tableAttr.nullable) {
+          throw new AnalysisException(
+            s"Cannot write nullable values to non-null column '${tableAttr.name}'")
+        }
+
+        val errors = new mutable.ArrayBuffer[String]()
+        val canWrite = DataType.canWrite(

Review comment:
       Important: I am passing `byName = false`. We can pass `byName = true` if we wanted to throw an exception when a struct with fields (a, b) is assigned as a struct with fields (b, a). However, that would make it different compared to SQL INSERTs.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549641751



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>

Review comment:
       Done.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549538579



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))

Review comment:
       This assumes that the rewrite was needed, but we expect the `MergeIntoTable` plan to be resolved after this rule runs. As a result, this rule will always produce a new `MergeIntoTable` instance and `fastEquals` used by the rule executor will always compare the full tree.
   
   This shouldn't affect correctness, but I think it is a best practice to detect whether anything has changed and avoid creating a new node if it is the same.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r549642325



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val namePartsSeq = assignments.map(_.key).map(getNameParts)
+          namePartsSeq.foreach { nameParts =>
+            if (nameParts.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${nameParts.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = namePartsSeq.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              "Duplicate column names inside INSERT clause: " +
+              duplicateColNames.mkString("[", ",", "]"))
+          }
+
+          // reorder assignments by the target table column order
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, colNames, assignments))
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      insertCols: Seq[String],
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+    targetTable.output.map { targetAttr =>
+      val assignment = assignments.find(a => resolver(targetAttr.name, getNameParts(a.key).head))
+      if (assignment.isEmpty) {
+        throw new AnalysisException(
+          s"Cannot find column '${targetAttr.name}' of the target table among " +
+          s"the INSERT columns: ${insertCols.mkString(", ")}. " +
+          "INSERT clauses must provide values for all columns of the target table.")
+      }
+
+      val key = assignment.get.key
+      val value = assignment.get.value
+      Assignment(key, castIfNeeded(value, key.dataType, resolver))

Review comment:
       Updated.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552123828



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))
+      }
+    }
+  }
+
+  private def applyStructUpdates(
+      col: NamedExpression,
+      updates: Seq[ColumnUpdate],
+      fields: Seq[StructField],
+      resolver: Resolver,
+      namePrefix: Seq[String]): Expression = {
+
+    // build field expressions
+    val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) =>
+      Alias(GetStructField(col, ordinal, Some(field.name)), field.name)()
+    }
+
+    // recursively apply this method on nested fields
+    val newUpdates = updates.map(u => u.copy(ref = u.ref.tail))
+    val updatedFieldExprs = applyUpdates(fieldExprs, newUpdates, resolver, namePrefix :+ col.name)
+
+    // construct a new struct with updated field expressions
+    toNamedStruct(fields, updatedFieldExprs)
+  }
+
+  private def toNamedStruct(fields: Seq[StructField], fieldExprs: Seq[Expression]): Expression = {
+    val namedStructExprs = fields.zip(fieldExprs).flatMap { case (field, expr) =>
+      Seq(Literal(field.name), expr)
+    }
+    CreateNamedStruct(namedStructExprs)
+  }
+
+  private def hasExactMatch(
+      updates: Seq[ColumnUpdate],
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    updates.exists(assignment => isExactMatch(assignment, col, resolver))
+  }
+
+  private def isExactMatch(
+      update: ColumnUpdate,
+      col: NamedExpression,
+      resolver: Resolver): Boolean = {
+
+    update.ref match {
+      case Seq(namePart) if resolver(namePart, col.name) => true
+      case _ => false
+    }
+  }
+
+  protected def castIfNeeded(
+      tableAttr: NamedExpression,
+      expr: Expression,
+      resolver: Resolver): Expression = {
+
+    val storeAssignmentPolicy = conf.storeAssignmentPolicy
+
+    // run the type check and catch type errors
+    storeAssignmentPolicy match {
+      case StoreAssignmentPolicy.STRICT | StoreAssignmentPolicy.ANSI =>
+        if (expr.nullable && !tableAttr.nullable) {
+          throw new AnalysisException(
+            s"Cannot write nullable values to non-null column '${tableAttr.name}'")
+        }
+
+        val errors = new mutable.ArrayBuffer[String]()
+        val canWrite = DataType.canWrite(

Review comment:
       I'm not sure about this one either. `byName = true` is more careful, so I would probably go with that to start with.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552121671



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.spark.sql.AnalysisException;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class TestMerge extends SparkRowLevelOperationsTestBase {
+
+  public TestMerge(String catalogName, String implementation, Map<String, String> config,
+                   String fileFormat, boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS source");
+  }
+
+  // TODO: tests for reordering when operations succeed (both insert and update actions)
+  // TODO: tests for modifying fields in a null struct

Review comment:
       Okay, so the struct gets created and unspecified fields get set to null. Sounds good to me.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r551862968



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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, AnsiCast, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import scala.collection.mutable
+
+trait AssignmentAlignmentSupport extends CastSupport {
+
+  private case class ColumnUpdate(ref: Seq[String], expr: Expression)
+
+  /**
+   * Aligns assignments to match table columns.
+   * <p>
+   * This method processes and reorders given assignments so that each target column gets
+   * an expression it should be set to. If a column does not have a matching assignment,
+   * it will be set to its current value. For example, if one passes a table with columns c1, c2
+   * and an assignment c2 = 1, this method will return c1 = c1, c2 = 1.
+   * <p>
+   * This method also handles updates to nested columns. If there is an assignment to a particular
+   * nested field, this method will construct a new struct with one field updated
+   * preserving other fields that have not been modified. For example, if one passes a table with
+   * columns c1, c2 where c2 is a struct with fields n1 and n2 and an assignment c2.n2 = 1,
+   * this method will return c1 = c1, c2 = struct(c2.n1, 1).
+   *
+   * @param table a target table
+   * @param assignments assignments to align
+   * @return aligned assignments that match table columns
+   */
+  protected def alignAssignments(
+      table: LogicalPlan,
+      assignments: Seq[Assignment]): Seq[Assignment] = {
+
+    val columnUpdates = assignments.map(a => ColumnUpdate(a.key, a.value))
+    val outputExprs = applyUpdates(table.output, columnUpdates)
+    outputExprs.zip(table.output).map {
+      case (expr, attr) => Assignment(attr, expr)
+    }
+  }
+
+  private def applyUpdates(
+      cols: Seq[NamedExpression],
+      updates: Seq[ColumnUpdate],
+      resolver: Resolver = conf.resolver,
+      namePrefix: Seq[String] = Nil): Seq[Expression] = {
+
+    // iterate through columns at the current level and find which column updates match
+    cols.map { col =>
+      // find matches for this column or any of its children
+      val prefixMatchedUpdates = updates.filter(a => resolver(a.ref.head, col.name))
+      prefixMatchedUpdates match {
+        // if there is no exact match and no match for children, return the column as is
+        case updates if updates.isEmpty =>
+          col
+
+        // if there is an exact match, return the assigned expression
+        case Seq(update) if isExactMatch(update, col, resolver) =>
+          castIfNeeded(col, update.expr, resolver)
+
+        // if there are matches only for children
+        case updates if !hasExactMatch(updates, col, resolver) =>
+          col.dataType match {
+            case StructType(fields) =>
+              applyStructUpdates(col, prefixMatchedUpdates, fields, resolver, namePrefix)
+            case otherType =>
+              val colName = (namePrefix :+ col.name).mkString(".")
+              throw new AnalysisException(
+                "Updating nested fields is only supported for StructType " +
+                s"but $colName is of type $otherType"
+              )
+          }
+
+        // if there are conflicting updates, throw an exception
+        // there are two illegal scenarios:
+        // - multiple updates to the same column
+        // - updates to a top-level struct and its nested fields (e.g., a.b and a.b.c)
+        case updates if hasExactMatch(updates, col, resolver) =>
+          val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString("."))
+          throw new AnalysisException(
+            "Updates are in conflict for these columns: " +
+            conflictingCols.distinct.mkString("[", ", ", "]"))

Review comment:
       Done.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#issuecomment-755823931


   Looks good overall. I'll merge this even with test TODO items so that we can get working on other MERGE tasks. Thanks, @aokolnychyi!


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1986: Spark: Add a rule to align updates in MERGE operations

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1986:
URL: https://github.com/apache/iceberg/pull/1986#discussion_r552122886



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignMergeIntoTable.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.plans.logical.{Assignment, DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, UpdateAction}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+
+case class AlignMergeIntoTable(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m: MergeIntoTable if m.resolved =>
+      val alignedMatchedActions = m.matchedActions.map {
+        case u @ UpdateAction(_, assignments) =>
+          u.copy(assignments = alignAssignments(m.targetTable, assignments))
+        case d: DeleteAction => d
+      }
+
+      val alignedNotMatchedActions = m.notMatchedActions.map {
+        case i @ InsertAction(_, assignments) =>
+          // check no nested columns are present
+          val refs = assignments.map(_.key).map(asAssignmentReference)
+          refs.foreach { ref =>
+            if (ref.size > 1) {
+              throw new AnalysisException(
+                "Nested fields are not supported inside INSERT clauses of MERGE operations: " +
+                s"${ref.mkString("`", "`.`", "`")}")
+            }
+          }
+
+          val colNames = refs.map(_.head)
+
+          // check there are no duplicates
+          val duplicateColNames = colNames.groupBy(identity).collect {
+            case (name, matchingNames) if matchingNames.size > 1 => name
+          }
+
+          if (duplicateColNames.nonEmpty) {
+            throw new AnalysisException(
+              s"Duplicate column names inside INSERT clause: ${duplicateColNames.mkString(", ")}")
+          }
+
+          // reorder assignments by the target table column order
+          val assignmentMap = colNames.zip(assignments).toMap
+          i.copy(assignments = alignInsertActionAssignments(m.targetTable, assignmentMap))
+
+        case _ =>
+          throw new AnalysisException("Not matched actions can only contain INSERT")
+      }
+
+      m.copy(matchedActions = alignedMatchedActions, notMatchedActions = alignedNotMatchedActions)
+  }
+
+  private def alignInsertActionAssignments(
+      targetTable: LogicalPlan,
+      assignmentMap: Map[String, Assignment]): Seq[Assignment] = {
+
+    val resolver = conf.resolver
+
+    targetTable.output.map { targetAttr =>
+      val assignment = assignmentMap
+        .find { case (name, _) => resolver(name, targetAttr.name) }
+        .map { case (_, assignment) => assignment }
+
+      if (assignment.isEmpty) {
+        throw new AnalysisException(

Review comment:
       Sounds good.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org