You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/09/17 23:30:52 UTC

[GitHub] [spark] fqaiser94 opened a new pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

fqaiser94 opened a new pull request #29795:
URL: https://github.com/apache/spark/pull/29795


   Currently a WIP, I will add details once its officially ready for review!
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r495620107



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.addCase("Nullable StructTypes") { _ =>
+        nullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.run()
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    val maxDepth = 20
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 5,
+      numsToAdd = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToDrop = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToAdd = 10 to 14,
+      numsToDrop = 5 to 9)
+  }
+}
+
+class UpdateFieldsBenchmark extends QueryTest with SharedSparkSession {

Review comment:
       I removed some unnecessary tests but I would prefer to keep the rest as the methods being tested are not entirely trivial (e.g. recursion is involved). 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694554861






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698688247


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33719/
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492435473



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       IIUC you mean put `CombineUpdateFields` in a separate batch that runs before the `SimplifyExtractValueOps` batch and remove the `ultimateStruct` method?
   
   If so, we'll miss out on some optimizations because after `SimplifyExtractValueOps` runs we might again end up with code containing `UpdateFields(UpdateFields(_, _))`. The `simplify add multiple nested fields to struct` test in `complexTypesSuite` is a good example of a test that will fail in this scenario. 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       good to know, cheers

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       oh I see, passes my tests so I have made the change.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(

Review comment:
       I created another suite called `UpdateFieldsPerformanceSuite` as part of this PR where I reuse this function. 
   If you want, I could put the contents of `UpdateFieldsPerformanceSuite` in `ColumnExpressionSuite` and then I would be able to keep this function (`checkAnswer`) in just `ColumnExpressionSuite`. Just let me know. 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695896040






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010729






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #29795:
URL: https://github.com/apache/spark/pull/29795


   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r493064492



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(

Review comment:
       I created another suite called `UpdateFieldsPerformanceSuite` as part of this PR where I reuse this function. 
   If you want, I could put the contents of `UpdateFieldsPerformanceSuite` in `ColumnExpressionSuite` and then I would be able to keep this function (`checkAnswer`) in just `ColumnExpressionSuite`. Just let me know. 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492559088



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       I mean something like
   ```
   case GetStructField(updateFields: UpdateFields, ordinal, _) if !updateFields.structExpr.isInstanceOf[UpdateFields] =>
     ...
   ```
   As we know that there must be a rule to merge adjacent `UpdateFields` and this rule can wait for it to happen. They are still in the same batch.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699698343






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r493196364



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {

Review comment:
       nit: 4 space indentation

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {
+    if (depths.length == 1) {
+      StructType(colNums.map { colNum =>
+        StructField(nestedColName(depths.head, colNum), IntegerType, nullable = false)
+      })
+    } else {
+      val depth = depths.head
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(depths.tail, colNums, nullable)
+          structFields :+ StructField(nestedColName(depth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          structFields :+ StructField(nestedColName(depth, colNum), IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(depths: Seq[Int], colNums: Seq[Int]): Row = {
+    if (depths.length == 1) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 => values :+ nestedRow(depths.tail, colNums)
+        case (values, colNum) => values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param depth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  private def nestedDf(
+    depth: Int, numColsAtEachDepth: Int, nullable: Boolean = false): DataFrame = {

Review comment:
       ditto

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {
+    if (depths.length == 1) {
+      StructType(colNums.map { colNum =>
+        StructField(nestedColName(depths.head, colNum), IntegerType, nullable = false)
+      })
+    } else {
+      val depth = depths.head
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(depths.tail, colNums, nullable)
+          structFields :+ StructField(nestedColName(depth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          structFields :+ StructField(nestedColName(depth, colNum), IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(depths: Seq[Int], colNums: Seq[Int]): Row = {
+    if (depths.length == 1) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 => values :+ nestedRow(depths.tail, colNums)
+        case (values, colNum) => values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param depth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  private def nestedDf(
+    depth: Int, numColsAtEachDepth: Int, nullable: Boolean = false): DataFrame = {
+    require(depth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val depths = 1 to depth
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(depths, colNums)
+    val nestedColumnDataType = nestedStructType(depths, colNums, nullable)
+
+    spark.createDataFrame(
+      sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  test("nestedDf should generate nested DataFrames") {
+    checkAnswer(
+      nestedDf(1, 1),
+      Row(Row(0)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(1, 2),
+      Row(Row(0, 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", IntegerType, nullable = false),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 1),
+      Row(Row(Row(0))) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false))),
+          nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 2),
+      Row(Row(Row(0, 1), 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false),
+          StructField("nested2Col1", IntegerType, nullable = false))),
+          nullable = false),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 2, nullable = true),
+      Row(Row(Row(0, 1), 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false),
+          StructField("nested2Col1", IntegerType, nullable = false))),
+          nullable = true),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = true))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  private def addDropNestedColumns(
+    column: Column,

Review comment:
       ditto

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {

Review comment:
       if we care about performance, shall we turn it into a benchmark like `OrcReadBenchmark`?




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010729






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r497050874



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.addCase("Nullable StructTypes") { _ =>
+        nullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.run()
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    val maxDepth = 20
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 5,
+      numsToAdd = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToDrop = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToAdd = 10 to 14,
+      numsToDrop = 5 to 9)
+  }
+}
+
+class UpdateFieldsBenchmark extends QueryTest with SharedSparkSession {

Review comment:
       Sure, 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r495623163



##########
File path: sql/core/benchmarks/UpdateFieldsBenchmark-results.txt
##########
@@ -0,0 +1,26 @@
+================================================================================================
+Add 2 columns and drop 2 columns at 3 different depths of nesting
+================================================================================================
+
+OpenJDK 64-Bit Server VM 1.8.0_212-b03 on Mac OS X 10.14.6
+Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz
+Add 2 columns and drop 2 columns at 3 different depths of nesting:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+-------------------------------------------------------------------------------------------------------------------------------------------------
+To non-nullable StructTypes using performant method                           10             11           2          0.0      Infinity       1.0X
+To nullable StructTypes using performant method                                9             10           1          0.0      Infinity       1.0X
+To non-nullable StructTypes using non-performant method                     2457           2464          10          0.0      Infinity       0.0X
+To nullable StructTypes using non-performant method                        42641          43804        1644          0.0      Infinity       0.0X

Review comment:
       As expected, this last result isn't great (43 seconds). 
   It's partially because of the non-performant method and partially because the optimizer rules aren't able to perfectly optimize complex nullable StructType scenarios (I've documented these scenarios in this [commit](https://github.com/apache/spark/pull/29795/commits/4fe48b4287c81e73276165453477811211e341d9)). 
   It should be possible to improve the optimizer rules further in the future. I have a couple of simple ideas I'm toying around with but it will take me a while to reason/test if they are safe from a correctness point of view. 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701016205


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33868/
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698688257






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698688261


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33719/
   Test FAILed.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492186072



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       using `semanticEquals` is safer

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       shall we apply this rule after `UpdateFields` are all merged? then we don't need to do `ultimateStruct`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *

Review comment:
       I think the same issue happens in `withColumn` as well. I'm fine with method doc.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(

Review comment:
       If it's only used in one test suite, let's move it there. We can move it to `QueryTest` if we see more and more suites using 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854450






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698688257






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r493063165



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       oh I see, passes my tests so I have made the 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-700986057


   **[Test build #129251 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129251/testReport)** for PR 29795 at commit [`7e51f35`](https://github.com/apache/spark/commit/7e51f35580db72fda11153d76caf232b83e617cd).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701005276


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33868/
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010182






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-700986057


   **[Test build #129251 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129251/testReport)** for PR 29795 at commit [`7e51f35`](https://github.com/apache/spark/commit/7e51f35580db72fda11153d76caf232b83e617cd).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698746139






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699732585


   **[Test build #129156 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129156/testReport)** for PR 29795 at commit [`cca6f37`](https://github.com/apache/spark/commit/cca6f37d03df6c41561dc2b4cc127ebf1305a8ed).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492189437



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *

Review comment:
       I think the same issue happens in `withColumn` as well. I'm fine with method doc.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698688257






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-696016893


   cc @cloud-fan @dbtsai @maropu @viirya


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854224


   **[Test build #128923 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128923/testReport)** for PR 29795 at commit [`2f16213`](https://github.com/apache/spark/commit/2f16213ea0a658f481f8f745d759a813fc844dfc).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695896040






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494698625



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {
+      values.map {
+        case (field, _) if resolver(field.name, name) => newFieldExpr
+        case x => x
+      }
+    } else {
+      values :+ newFieldExpr
+    }
+  }
+
+  override def children: Seq[Expression] = valExpr :: Nil
+
+  override def dataType: DataType = throw new UnresolvedException(this, "dataType")

Review comment:
       done

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {

Review comment:
       thanks for sharing the code, done

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {
+      values.map {
+        case (field, _) if resolver(field.name, name) => newFieldExpr
+        case x => x
+      }
+    } else {
+      values :+ newFieldExpr
+    }
+  }
+
+  override def children: Seq[Expression] = valExpr :: Nil
+
+  override def dataType: DataType = throw new UnresolvedException(this, "dataType")
+
+  override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
+
+  override def prettyName: String = "WithField"
+}
+
+/**
+ * Drop a field by name.
+ */
+case class DropField(name: String) extends StructFieldsOperation {
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] =
+    values.filterNot { case (field, _) => resolver(field.name, name) }
+}
+
+/**
+ * Updates fields in a struct.
+ */
+case class UpdateFields(structExpr: Expression, fieldOps: Seq[StructFieldsOperation])
+  extends Unevaluable {
 
   override def checkInputDataTypes(): TypeCheckResult = {
-    if (!structExpr.dataType.isInstanceOf[StructType]) {
-      TypeCheckResult.TypeCheckFailure(
-        "struct argument should be struct type, got: " + structExpr.dataType.catalogString)
+    val dataType = structExpr.dataType
+    if (!dataType.isInstanceOf[StructType]) {
+      TypeCheckResult.TypeCheckFailure("struct argument should be struct type, got: " +
+        dataType.catalogString)
+    } else if (newExprs.isEmpty) {
+      TypeCheckResult.TypeCheckFailure("cannot drop all fields in struct")
     } else {
       TypeCheckResult.TypeCheckSuccess
     }
   }
 
-  override def children: Seq[Expression] = structExpr +: valExprs
+  override def children: Seq[Expression] = structExpr +: fieldOps.collect {
+    case e: Expression => e
+  }
 
-  override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType]
+  override def dataType: StructType = StructType(newFields)
 
   override def nullable: Boolean = structExpr.nullable
 
-  override def prettyName: String = "with_fields"
+  override def prettyName: String = "update_fields"
 
-  lazy val evalExpr: Expression = {
-    val existingExprs = structExpr.dataType.asInstanceOf[StructType].fieldNames.zipWithIndex.map {
-      case (name, i) => (name, GetStructField(KnownNotNull(structExpr), i).asInstanceOf[Expression])
+  private lazy val existingFieldExprs: Seq[(StructField, Expression)] =
+    structExpr.dataType.asInstanceOf[StructType].fields.zipWithIndex.map {
+      case (field, i) => (field, GetStructField(structExpr, i))
     }
 
-    val addOrReplaceExprs = names.zip(valExprs)
-
-    val resolver = SQLConf.get.resolver
-    val newExprs = addOrReplaceExprs.foldLeft(existingExprs) {
-      case (resultExprs, newExpr @ (newExprName, _)) =>
-        if (resultExprs.exists(x => resolver(x._1, newExprName))) {
-          resultExprs.map {
-            case (name, _) if resolver(name, newExprName) => newExpr
-            case x => x
-          }
-        } else {
-          resultExprs :+ newExpr
-        }
-    }.flatMap { case (name, expr) => Seq(Literal(name), expr) }
+  private lazy val newFieldExprs: Seq[(StructField, Expression)] =
+    fieldOps.foldLeft(existingFieldExprs)((exprs, op) => op(exprs))
 
-    val expr = CreateNamedStruct(newExprs)
-    if (structExpr.nullable) {
-      If(IsNull(structExpr), Literal(null, expr.dataType), expr)
-    } else {
-      expr
-    }
+  private lazy val newFields: Seq[StructField] = newFieldExprs.map(_._1)
+
+  lazy val newExprs: Seq[Expression] = newFieldExprs.map(_._2)
+
+  private lazy val createNamedStructExpr = CreateNamedStruct(newFieldExprs.flatMap {
+    case (field, expr) => Seq(Literal(field.name), expr)
+  })
+
+  lazy val evalExpr: Expression = if (structExpr.nullable) {

Review comment:
       done

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.

Review comment:
       I've made this change but now that I think about it, I don't think its actually classifies as a "noop". We still reconstruct the struct unfortunately e.g.
   ```
   val structType = StructType(Seq(
       StructField("a", IntegerType, nullable = false),
       StructField("b", IntegerType, nullable = true),
       StructField("c", IntegerType, nullable = false)))
   
   val structLevel1: DataFrame = spark.createDataFrame(
       sparkContext.parallelize(Row(Row(1, null, 3)) :: Nil),
       StructType(Seq(StructField("a", structType, nullable = false))))
   
   structLevel1.withColumn("a", 'a.dropFields("d")).explain()
   
   == Physical Plan ==
   *(1) Project [named_struct(a, a#1.a, b, a#1.b, c, a#1.c) AS a#3]
   +- *(1) Scan ExistingRDD[a#1]
   ```
   Should I revert this?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("c"))
+   *   // result: {"a":1,"b":2}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b", "c"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("a", "b"))
+   *   // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct
+   *
+   *   val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: null of type struct<a:int>
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b"))
+   *   // result: {"a":{"a":1}}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.c"))
+   *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+   * }}}
+   *
+   * This method supports dropping multiple nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b", "a.c"))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * However, if you are going to drop multiple nested fields, it is more optimal to extract
+   * out the nested struct before dropping multiple fields from it e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * @group expr_ops
+   * @since 3.1.0
+   */
+  // scalastyle:on line.size.limit
+  def dropFields(fieldNames: String*): Column = withExpr {
+    def dropField(structExpr: Expression, fieldName: String): UpdateFields =
+      updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name))
+
+    fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) {
+      (resExpr, fieldName) => dropField(resExpr, fieldName)
+    }
+  }
+
+  private def nameParts(fieldName: String): Seq[String] = {
+    require(fieldName != null, "fieldName cannot be null")
+
+    if (fieldName.isEmpty) {
       fieldName :: Nil

Review comment:
       we've discussed this before [here](https://github.com/apache/spark/pull/27066#discussion_r448416127) :)
   Its needed for `withField` and I think we should support it in `dropFields` as well because `Dataset.drop` supports it: 
   ```
   scala> Seq((1, 2)).toDF("a", "").drop("").printSchema
   root
    |-- a: integer (nullable = false)
   ```
   I've added a test case to demonstrate this works on the `dropFields` side but otherwise left the code unchanged. 
   

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("c"))
+   *   // result: {"a":1,"b":2}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b", "c"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("a", "b"))
+   *   // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct
+   *
+   *   val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: null of type struct<a:int>
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b"))
+   *   // result: {"a":{"a":1}}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.c"))
+   *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+   * }}}
+   *
+   * This method supports dropping multiple nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b", "a.c"))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * However, if you are going to drop multiple nested fields, it is more optimal to extract
+   * out the nested struct before dropping multiple fields from it e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * @group expr_ops
+   * @since 3.1.0
+   */
+  // scalastyle:on line.size.limit
+  def dropFields(fieldNames: String*): Column = withExpr {
+    def dropField(structExpr: Expression, fieldName: String): UpdateFields =
+      updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name))
+
+    fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) {
+      (resExpr, fieldName) => dropField(resExpr, fieldName)
+    }
+  }
+
+  private def nameParts(fieldName: String): Seq[String] = {
+    require(fieldName != null, "fieldName cannot be null")
+
+    if (fieldName.isEmpty) {
       fieldName :: Nil
     } else {
       CatalystSqlParser.parseMultipartIdentifier(fieldName)
     }
-    withFieldHelper(expr, nameParts, Nil, col.expr)
   }
 
-  private def withFieldHelper(
-      struct: Expression,
-      namePartsRemaining: Seq[String],
-      namePartsDone: Seq[String],
-      value: Expression) : WithFields = {
-    val name = namePartsRemaining.head
+  private def updateFieldsHelper(
+    structExpr: Expression,

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(
+    df: => DataFrame,
+    expectedAnswer: Seq[Row],

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {
+    if (depths.length == 1) {
+      StructType(colNums.map { colNum =>
+        StructField(nestedColName(depths.head, colNum), IntegerType, nullable = false)
+      })
+    } else {
+      val depth = depths.head
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(depths.tail, colNums, nullable)
+          structFields :+ StructField(nestedColName(depth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          structFields :+ StructField(nestedColName(depth, colNum), IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(depths: Seq[Int], colNums: Seq[Int]): Row = {
+    if (depths.length == 1) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 => values :+ nestedRow(depths.tail, colNums)
+        case (values, colNum) => values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param depth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  private def nestedDf(
+    depth: Int, numColsAtEachDepth: Int, nullable: Boolean = false): DataFrame = {

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {
+    if (depths.length == 1) {
+      StructType(colNums.map { colNum =>
+        StructField(nestedColName(depths.head, colNum), IntegerType, nullable = false)
+      })
+    } else {
+      val depth = depths.head
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(depths.tail, colNums, nullable)
+          structFields :+ StructField(nestedColName(depth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          structFields :+ StructField(nestedColName(depth, colNum), IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(depths: Seq[Int], colNums: Seq[Int]): Row = {
+    if (depths.length == 1) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 => values :+ nestedRow(depths.tail, colNums)
+        case (values, colNum) => values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param depth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  private def nestedDf(
+    depth: Int, numColsAtEachDepth: Int, nullable: Boolean = false): DataFrame = {
+    require(depth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val depths = 1 to depth
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(depths, colNums)
+    val nestedColumnDataType = nestedStructType(depths, colNums, nullable)
+
+    spark.createDataFrame(
+      sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  test("nestedDf should generate nested DataFrames") {
+    checkAnswer(
+      nestedDf(1, 1),
+      Row(Row(0)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(1, 2),
+      Row(Row(0, 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", IntegerType, nullable = false),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 1),
+      Row(Row(Row(0))) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false))),
+          nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 2),
+      Row(Row(Row(0, 1), 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false),
+          StructField("nested2Col1", IntegerType, nullable = false))),
+          nullable = false),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 2, nullable = true),
+      Row(Row(Row(0, 1), 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false),
+          StructField("nested2Col1", IntegerType, nullable = false))),
+          nullable = true),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = true))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  private def addDropNestedColumns(
+    column: Column,

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {

Review comment:
       done, please take a closer look at this as this was relatively new 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010729






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r493187951



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {
+      values.map {
+        case (field, _) if resolver(field.name, name) => newFieldExpr
+        case x => x
+      }
+    } else {
+      values :+ newFieldExpr
+    }
+  }
+
+  override def children: Seq[Expression] = valExpr :: Nil
+
+  override def dataType: DataType = throw new UnresolvedException(this, "dataType")

Review comment:
       This is not really unresolved. If we don't expect to reach here, probably `IllegalStateException` is better.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {

Review comment:
       nit: can we avoid iterating the `values` twice?
   ```
   val result = ArrayBuffer.empty[(StructField, Expression)]
   val newFieldExpr = ...
   var hasMatch = false
   for((field, expr) <- values) {
     if (resolver(field.name, name)) {
       hasMatch = true
       ...
     } else ...
   }
   if (!hasMatch) result += newFieldExpr
   result
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {

Review comment:
       nit: can we avoid iterating the `values` twice?
   ```
   val result = ArrayBuffer.empty[(StructField, Expression)]
   val newFieldExpr = ...
   var hasMatch = false
   for((field, expr) <- values) {
     if (resolver(field.name, name)) {
       hasMatch = true
       ...
     } else ...
   }
   if (!hasMatch) result += newFieldExpr
   result.toArray
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {
+      values.map {
+        case (field, _) if resolver(field.name, name) => newFieldExpr
+        case x => x
+      }
+    } else {
+      values :+ newFieldExpr
+    }
+  }
+
+  override def children: Seq[Expression] = valExpr :: Nil
+
+  override def dataType: DataType = throw new UnresolvedException(this, "dataType")
+
+  override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
+
+  override def prettyName: String = "WithField"
+}
+
+/**
+ * Drop a field by name.
+ */
+case class DropField(name: String) extends StructFieldsOperation {
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] =
+    values.filterNot { case (field, _) => resolver(field.name, name) }
+}
+
+/**
+ * Updates fields in a struct.
+ */
+case class UpdateFields(structExpr: Expression, fieldOps: Seq[StructFieldsOperation])
+  extends Unevaluable {
 
   override def checkInputDataTypes(): TypeCheckResult = {
-    if (!structExpr.dataType.isInstanceOf[StructType]) {
-      TypeCheckResult.TypeCheckFailure(
-        "struct argument should be struct type, got: " + structExpr.dataType.catalogString)
+    val dataType = structExpr.dataType
+    if (!dataType.isInstanceOf[StructType]) {
+      TypeCheckResult.TypeCheckFailure("struct argument should be struct type, got: " +
+        dataType.catalogString)
+    } else if (newExprs.isEmpty) {
+      TypeCheckResult.TypeCheckFailure("cannot drop all fields in struct")
     } else {
       TypeCheckResult.TypeCheckSuccess
     }
   }
 
-  override def children: Seq[Expression] = structExpr +: valExprs
+  override def children: Seq[Expression] = structExpr +: fieldOps.collect {
+    case e: Expression => e
+  }
 
-  override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType]
+  override def dataType: StructType = StructType(newFields)
 
   override def nullable: Boolean = structExpr.nullable
 
-  override def prettyName: String = "with_fields"
+  override def prettyName: String = "update_fields"
 
-  lazy val evalExpr: Expression = {
-    val existingExprs = structExpr.dataType.asInstanceOf[StructType].fieldNames.zipWithIndex.map {
-      case (name, i) => (name, GetStructField(KnownNotNull(structExpr), i).asInstanceOf[Expression])
+  private lazy val existingFieldExprs: Seq[(StructField, Expression)] =
+    structExpr.dataType.asInstanceOf[StructType].fields.zipWithIndex.map {
+      case (field, i) => (field, GetStructField(structExpr, i))
     }
 
-    val addOrReplaceExprs = names.zip(valExprs)
-
-    val resolver = SQLConf.get.resolver
-    val newExprs = addOrReplaceExprs.foldLeft(existingExprs) {
-      case (resultExprs, newExpr @ (newExprName, _)) =>
-        if (resultExprs.exists(x => resolver(x._1, newExprName))) {
-          resultExprs.map {
-            case (name, _) if resolver(name, newExprName) => newExpr
-            case x => x
-          }
-        } else {
-          resultExprs :+ newExpr
-        }
-    }.flatMap { case (name, expr) => Seq(Literal(name), expr) }
+  private lazy val newFieldExprs: Seq[(StructField, Expression)] =
+    fieldOps.foldLeft(existingFieldExprs)((exprs, op) => op(exprs))
 
-    val expr = CreateNamedStruct(newExprs)
-    if (structExpr.nullable) {
-      If(IsNull(structExpr), Literal(null, expr.dataType), expr)
-    } else {
-      expr
-    }
+  private lazy val newFields: Seq[StructField] = newFieldExprs.map(_._1)
+
+  lazy val newExprs: Seq[Expression] = newFieldExprs.map(_._2)
+
+  private lazy val createNamedStructExpr = CreateNamedStruct(newFieldExprs.flatMap {
+    case (field, expr) => Seq(Literal(field.name), expr)
+  })
+
+  lazy val evalExpr: Expression = if (structExpr.nullable) {

Review comment:
       nit: use local variable if possible
   ```
   lazy val evalExpr: Expression = {
     val createNamedStructExpr = ...
     if (structExpr.nullable) ...
   }
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.

Review comment:
       Let's explicitly mention that, if the name doesn't match any field, it's noop.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("c"))
+   *   // result: {"a":1,"b":2}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b", "c"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("a", "b"))
+   *   // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct
+   *
+   *   val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: null of type struct<a:int>
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b"))
+   *   // result: {"a":{"a":1}}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.c"))
+   *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+   * }}}
+   *
+   * This method supports dropping multiple nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b", "a.c"))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * However, if you are going to drop multiple nested fields, it is more optimal to extract
+   * out the nested struct before dropping multiple fields from it e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * @group expr_ops
+   * @since 3.1.0
+   */
+  // scalastyle:on line.size.limit
+  def dropFields(fieldNames: String*): Column = withExpr {
+    def dropField(structExpr: Expression, fieldName: String): UpdateFields =
+      updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name))
+
+    fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) {
+      (resExpr, fieldName) => dropField(resExpr, fieldName)
+    }
+  }
+
+  private def nameParts(fieldName: String): Seq[String] = {
+    require(fieldName != null, "fieldName cannot be null")
+
+    if (fieldName.isEmpty) {
       fieldName :: Nil

Review comment:
       shall we fail here? I don't think the field name can be an empty string.
   
   It's not related to this PR and we can consider it later.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("c"))
+   *   // result: {"a":1,"b":2}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b", "c"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("a", "b"))
+   *   // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct
+   *
+   *   val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: null of type struct<a:int>
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b"))
+   *   // result: {"a":{"a":1}}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.c"))
+   *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+   * }}}
+   *
+   * This method supports dropping multiple nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b", "a.c"))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * However, if you are going to drop multiple nested fields, it is more optimal to extract
+   * out the nested struct before dropping multiple fields from it e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * @group expr_ops
+   * @since 3.1.0
+   */
+  // scalastyle:on line.size.limit
+  def dropFields(fieldNames: String*): Column = withExpr {
+    def dropField(structExpr: Expression, fieldName: String): UpdateFields =
+      updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name))
+
+    fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) {
+      (resExpr, fieldName) => dropField(resExpr, fieldName)
+    }
+  }
+
+  private def nameParts(fieldName: String): Seq[String] = {
+    require(fieldName != null, "fieldName cannot be null")
+
+    if (fieldName.isEmpty) {
       fieldName :: Nil
     } else {
       CatalystSqlParser.parseMultipartIdentifier(fieldName)
     }
-    withFieldHelper(expr, nameParts, Nil, col.expr)
   }
 
-  private def withFieldHelper(
-      struct: Expression,
-      namePartsRemaining: Seq[String],
-      namePartsDone: Seq[String],
-      value: Expression) : WithFields = {
-    val name = namePartsRemaining.head
+  private def updateFieldsHelper(
+    structExpr: Expression,

Review comment:
       nit: 4 space indentation.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(
+    df: => DataFrame,
+    expectedAnswer: Seq[Row],

Review comment:
       nit: 4 space indentation.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010182


   **[Test build #128995 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128995/testReport)** for PR 29795 at commit [`650d366`](https://github.com/apache/spark/commit/650d366b71982ff496b6f57af66fcf82d77603bf).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-696016893


   cc @cloud-fan @dbtsai @maropu @viirya


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-704128351


   thanks, merging to master!


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492186072



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       using `semanticEquals` is safer




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r491753622



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
##########
@@ -537,18 +662,75 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper {
       query(testStructRelation),
       testStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           Literal(2) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           Literal(3) as "struct3B"))
 
     checkRule(
       query(testNullableStructRelation),
       testNullableStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(3)) as "struct3B"))
   }
+
+  test("simplify add multiple nested fields to struct") {
+    // this scenario is possible if users add multiple nested columns via the Column.withField API
+    // ideally, users should not be doing this.
+    val nullableStructLevel2 = LocalRelation(
+      'a1.struct(
+        'a2.struct('a3.int)).withNullability(false))
+
+    val query = {
+      val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2",
+        UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2)))))))
+
+      nullableStructLevel2.select(
+        UpdateFields(
+          addB3toA1A2,
+          Seq(WithField("a2", UpdateFields(
+            GetStructField(addB3toA1A2, 0), Seq(WithField("c3", Literal(3))))))).as("a1"))
+    }
+
+    val expected = nullableStructLevel2.select(
+      UpdateFields('a1, Seq(
+        // scalastyle:off line.size.limit
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: Nil)),
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil))
+        // scalastyle:on line.size.limit
+      )).as("a1"))
+
+    checkRule(query, expected)
+  }
+
+  test("simplify drop multiple nested fields in struct") {
+    // this scenario is possible if users drop multiple nested columns via the Column.dropFields API
+    // ideally, users should not be doing this.
+    val df = LocalRelation(
+      'a1.struct(
+        'a2.struct('a3.int, 'b3.int, 'c3.int).withNullability(false)
+      ).withNullability(false))
+
+    val query = {
+      val dropA1A2B = UpdateFields('a1, Seq(WithField("a2", UpdateFields(
+        GetStructField('a1, 0), Seq(DropField("b3"))))))
+
+      df.select(
+        UpdateFields(
+          dropA1A2B,
+          Seq(WithField("a2", UpdateFields(
+            GetStructField(dropA1A2B, 0), Seq(DropField("c3")))))).as("a1"))
+    }
+
+    val expected = df.select(
+      UpdateFields('a1, Seq(
+        WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3")))),
+        WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3"), DropField("c3"))))
+      )).as("a1"))

Review comment:
       This first `WithField` in here is entirely redundant as well and ideally we would optimize this away as well.
   However, in the interests of keeping this PR simple, I have opted to forgo writing any such optimizer rule.
   If necessary, we can address this in a future PR.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
##########
@@ -1514,27 +1516,578 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
       StructType(Seq(StructField("a", structType, nullable = true))))
 
     // extract newly added field
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("d", lit(4)).getField("d")),
       Row(4) :: Row(null) :: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
 
     // extract newly replaced field
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("a", lit(4)).getField("a")),
       Row(4) :: Row(null):: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
 
     // add new field, extract another field from original struct
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("d", lit(4)).getField("c")),
       Row(3) :: Row(null):: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
 
     // replace field, extract another field from original struct
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("a", lit(4)).getField("c")),
       Row(3) :: Row(null):: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
   }
+
+
+  test("dropFields should throw an exception if called on a non-StructType column") {
+    intercept[AnalysisException] {
+      testData.withColumn("key", $"key".dropFields("a"))
+    }.getMessage should include("struct argument should be struct type, got: int")
+  }
+
+  test("dropFields should throw an exception if fieldName argument is null") {
+    intercept[IllegalArgumentException] {
+      structLevel1.withColumn("a", $"a".dropFields(null))
+    }.getMessage should include("fieldName cannot be null")
+  }
+
+  test("dropFields should throw an exception if any intermediate structs don't exist") {
+    intercept[AnalysisException] {
+      structLevel2.withColumn("a", 'a.dropFields("x.b"))
+    }.getMessage should include("No such struct field x in a")
+
+    intercept[AnalysisException] {
+      structLevel3.withColumn("a", 'a.dropFields("a.x.b"))
+    }.getMessage should include("No such struct field x in a")
+  }
+
+  test("dropFields should throw an exception if intermediate field is not a struct") {
+    intercept[AnalysisException] {
+      structLevel1.withColumn("a", 'a.dropFields("b.a"))
+    }.getMessage should include("struct argument should be struct type, got: int")
+  }
+
+  test("dropFields should throw an exception if intermediate field reference is ambiguous") {
+    intercept[AnalysisException] {
+      val structLevel2: DataFrame = spark.createDataFrame(
+        sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil),
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", structType, nullable = false),
+            StructField("a", structType, nullable = false))),
+            nullable = false))))
+
+      structLevel2.withColumn("a", 'a.dropFields("a.b"))
+    }.getMessage should include("Ambiguous reference to fields")
+  }
+
+  test("dropFields should drop field in struct") {
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("b")),
+      Row(Row(1, 3)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in null struct") {
+    checkAnswer(
+      nullStructLevel1.withColumn("a", $"a".dropFields("b")),
+      Row(null) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = true))))
+  }
+
+  test("dropFields should drop multiple fields in struct") {
+    Seq(
+      structLevel1.withColumn("a", $"a".dropFields("b", "c")),
+      structLevel1.withColumn("a", 'a.dropFields("b").dropFields("c"))
+    ).foreach { df =>
+      checkAnswer(
+        df,
+        Row(Row(1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should throw an exception if no fields will be left in struct") {
+    intercept[AnalysisException] {
+      structLevel1.withColumn("a", 'a.dropFields("a", "b", "c"))
+    }.getMessage should include("cannot drop all fields in struct")
+  }
+
+  test("dropFields should drop field in nested struct") {
+    checkAnswer(
+      structLevel2.withColumn("a", 'a.dropFields("a.b")),
+      Row(Row(Row(1, 3))) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop multiple fields in nested struct") {
+    checkAnswer(
+      structLevel2.withColumn("a", 'a.dropFields("a.b", "a.c")),
+      Row(Row(Row(1))) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in nested null struct") {
+    checkAnswer(
+      nullStructLevel2.withColumn("a", $"a".dropFields("a.b")),
+      Row(Row(null)) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = true))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop multiple fields in nested null struct") {
+    checkAnswer(
+      nullStructLevel2.withColumn("a", $"a".dropFields("a.b", "a.c")),
+      Row(Row(null)) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = true))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in deeply nested struct") {
+    checkAnswer(
+      structLevel3.withColumn("a", 'a.dropFields("a.a.b")),
+      Row(Row(Row(Row(1, 3)))) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", StructType(Seq(
+              StructField("a", IntegerType, nullable = false),
+              StructField("c", IntegerType, nullable = false))),
+              nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop all fields with given name in struct") {
+    val structLevel1 = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(1, 2, 3)) :: Nil),
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("b")),
+      Row(Row(1)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in struct even if casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") {
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")),
+        Row(Row(1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("B", IntegerType, nullable = false))),
+            nullable = false))))
+
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")),
+        Row(Row(1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should not drop field in struct because casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")),
+        Row(Row(1, 1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("B", IntegerType, nullable = false))),
+            nullable = false))))
+
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")),
+        Row(Row(1, 1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("B", IntegerType, nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should drop nested field in struct even if casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") {
+      checkAnswer(
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a")),
+        Row(Row(Row(1), Row(1, 1))) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("A", StructType(Seq(
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false),
+            StructField("B", StructType(Seq(
+              StructField("a", IntegerType, nullable = false),
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false))),
+            nullable = false))))
+
+      checkAnswer(
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a")),
+        Row(Row(Row(1, 1), Row(1))) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", StructType(Seq(
+              StructField("a", IntegerType, nullable = false),
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false),
+            StructField("b", StructType(Seq(
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should throw an exception because casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
+      intercept[AnalysisException] {
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a"))
+      }.getMessage should include("No such struct field A in a, B")
+
+      intercept[AnalysisException] {
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a"))
+      }.getMessage should include("No such struct field b in a, B")
+    }
+  }
+
+  test("dropFields should drop only fields that exist") {
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("d")),
+      Row(Row(1, null, 3)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = true),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("b", "d")),
+      Row(Row(1, 3)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      structLevel2.withColumn("a", $"a".dropFields("a.b", "a.d")),
+      Row(Row(Row(1, 3))) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop multiple fields at arbitrary levels of nesting in a single call") {
+    val df: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil),
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", structType, nullable = false),
+          StructField("b", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      df.withColumn("a", $"a".dropFields("a.b", "b")),
+      Row(Row(Row(1, 3))) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))), nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields user-facing examples") {
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+        .select($"struct_col".dropFields("b")),
+      Row(Row(1)))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+        .select($"struct_col".dropFields("c")),
+      Row(Row(1, 2)))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+        .select($"struct_col".dropFields("b", "c")),
+      Row(Row(1)))
+
+    intercept[AnalysisException] {
+      sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+        .select($"struct_col".dropFields("a", "b"))
+    }.getMessage should include("cannot drop all fields in struct")
+
+    checkAnswer(
+      sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+        .select($"struct_col".dropFields("b")),
+      Row(null))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+        .select($"struct_col".dropFields("b")),
+      Row(Row(1)))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+        .select($"struct_col".dropFields("a.b")),
+      Row(Row(Row(1))))
+
+    intercept[AnalysisException] {
+      sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+        .select($"struct_col".dropFields("a.c"))
+    }.getMessage should include("Ambiguous reference to fields")
+
+    checkAnswer(
+      sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col")
+        .select($"struct_col".dropFields("a.b", "a.c")),
+      Row(Row(Row(1))))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col")
+        .select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c"))),
+      Row(Row(Row(1))))
+  }
+
+  test("should correctly handle different dropField + withField + getField combinations") {
+    val structType = StructType(Seq(
+      StructField("a", IntegerType, nullable = false),
+      StructField("b", IntegerType, nullable = false)))
+
+    val structLevel1: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(1, 2)) :: Nil),
+      StructType(Seq(StructField("a", structType, nullable = false))))
+
+    val nullStructLevel1: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(null) :: Nil),
+      StructType(Seq(StructField("a", structType, nullable = true))))
+
+    val nullableStructLevel1: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(1, 2)) :: Row(null) :: Nil),
+      StructType(Seq(StructField("a", structType, nullable = true))))
+
+    def check(
+      fieldOps: Column => Column,
+      getFieldName: String,
+      expectedValue: Option[Int]): Unit = {
+
+      def query(df: DataFrame): DataFrame =
+        df.select(fieldOps(col("a")).getField(getFieldName).as("res"))
+
+      checkAnswer(
+        query(structLevel1),
+        Row(expectedValue.orNull) :: Nil,
+        StructType(Seq(StructField("res", IntegerType, nullable = expectedValue.isEmpty))))
+
+      checkAnswer(
+        query(nullStructLevel1),
+        Row(null) :: Nil,
+        StructType(Seq(StructField("res", IntegerType, nullable = true))))
+
+      checkAnswer(
+        query(nullableStructLevel1),
+        Row(expectedValue.orNull) :: Row(null) :: Nil,
+        StructType(Seq(StructField("res", IntegerType, nullable = true))))
+    }
+
+    // add attribute, extract an attribute from the original struct
+    check(_.withField("c", lit(3)), "a", Some(1))
+    check(_.withField("c", lit(3)), "b", Some(2))
+
+    // add attribute, extract added attribute
+    check(_.withField("c", lit(3)), "c", Some(3))
+    check(_.withField("c", col("a.a")), "c", Some(1))
+    check(_.withField("c", col("a.b")), "c", Some(2))
+    check(_.withField("c", lit(null).cast(IntegerType)), "c", None)
+
+    // replace attribute, extract an attribute from the original struct
+    check(_.withField("b", lit(3)), "a", Some(1))
+    check(_.withField("a", lit(3)), "b", Some(2))
+
+    // replace attribute, extract replaced attribute
+    check(_.withField("b", lit(3)), "b", Some(3))
+    check(_.withField("b", lit(null).cast(IntegerType)), "b", None)
+    check(_.withField("a", lit(3)), "a", Some(3))
+    check(_.withField("a", lit(null).cast(IntegerType)), "a", None)
+
+    // drop attribute, extract an attribute from the original struct
+    check(_.dropFields("b"), "a", Some(1))
+    check(_.dropFields("a"), "b", Some(2))
+
+    // drop attribute, add attribute, extract an attribute from the original struct
+    check(_.dropFields("b").withField("c", lit(3)), "a", Some(1))
+    check(_.dropFields("a").withField("c", lit(3)), "b", Some(2))
+
+    // drop attribute, add another attribute, extract added attribute
+    check(_.dropFields("a").withField("c", lit(3)), "c", Some(3))
+    check(_.dropFields("b").withField("c", lit(3)), "c", Some(3))
+
+    // add attribute, drop attribute, extract an attribute from the original struct
+    check(_.withField("c", lit(3)).dropFields("a"), "b", Some(2))
+    check(_.withField("c", lit(3)).dropFields("b"), "a", Some(1))
+
+    // add attribute, drop another attribute, extract added attribute
+    check(_.withField("c", lit(3)).dropFields("a"), "c", Some(3))
+    check(_.withField("c", lit(3)).dropFields("b"), "c", Some(3))
+
+    // replace attribute, drop same attribute, extract an attribute from the original struct
+    check(_.withField("b", lit(3)).dropFields("b"), "a", Some(1))
+    check(_.withField("a", lit(3)).dropFields("a"), "b", Some(2))
+
+    // add attribute, drop same attribute, extract an attribute from the original struct
+    check(_.withField("c", lit(3)).dropFields("c"), "a", Some(1))
+    check(_.withField("c", lit(3)).dropFields("c"), "b", Some(2))
+
+    // add attribute, drop another attribute, extract added attribute
+    check(_.withField("b", lit(3)).dropFields("a"), "b", Some(3))
+    check(_.withField("a", lit(3)).dropFields("b"), "a", Some(3))
+    check(_.withField("b", lit(null).cast(IntegerType)).dropFields("a"), "b", None)
+    check(_.withField("a", lit(null).cast(IntegerType)).dropFields("b"), "a", None)
+
+    // drop attribute, add same attribute, extract added attribute
+    check(_.dropFields("b").withField("b", lit(3)), "b", Some(3))
+    check(_.dropFields("a").withField("a", lit(3)), "a", Some(3))
+    check(_.dropFields("b").withField("b", lit(null).cast(IntegerType)), "b", None)
+    check(_.dropFields("a").withField("a", lit(null).cast(IntegerType)), "a", None)
+    check(_.dropFields("c").withField("c", lit(3)), "c", Some(3))
+
+    // add attribute, drop same attribute, add same attribute again, extract added attribute
+    check(_.withField("c", lit(3)).dropFields("c").withField("c", lit(4)), "c", Some(4))
+  }
+
+  test("should move field up one level of nesting") {
+    val nullableStructLevel2: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(null)) :: Row(Row(Row(1, 2, 3))) :: Nil),
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", structType, nullable = true))),
+          nullable = true))))
+
+    // move a field up one level
+    checkAnswer(
+      nullableStructLevel2.select(
+        col("a").withField("b", col("a.a.b")).dropFields("a.b").as("res")),
+      Row(Row(null, null)) ::  Row(Row(Row(1, 3), 2)) :: Nil,
+      StructType(Seq(
+        StructField("res", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = true),
+          StructField("b", IntegerType, nullable = true))),
+          nullable = true))))
+
+    // move a field up one level and then extract it
+    checkAnswer(
+      nullableStructLevel2.select(col("a").withField("b", col("a.a.b")).getField("b").as("res")),
+      Row(null) :: Row(2) :: Nil,
+      StructType(Seq(StructField("res", IntegerType, nullable = true))))
+  }
+
+  test("should be able to refer to newly added nested column") {
+    intercept[AnalysisException] {
+      structLevel1.select($"a".withField("d", lit(4)).withField("e", $"a.d" + 1).as("a"))
+    }.getMessage should include("No such struct field d in a, b, c")
+
+    checkAnswer(
+      structLevel1
+        .select($"a".withField("d", lit(4)).as("a"))
+        .select($"a".withField("e", $"a.d" + 1).as("a")),
+      Row(Row(1, null, 3, 4, 5)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = true),
+          StructField("c", IntegerType, nullable = false),
+          StructField("d", IntegerType, nullable = false),
+          StructField("e", IntegerType, nullable = false))),
+          nullable = false))))
+  }

Review comment:
       I don't expect anyone will be surprised or feel that this is wrong but nevertheless, I did want to highlight this behaviour. Same goes for the two tests below. 

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
##########
@@ -537,18 +662,75 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper {
       query(testStructRelation),
       testStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           Literal(2) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           Literal(3) as "struct3B"))
 
     checkRule(
       query(testNullableStructRelation),
       testNullableStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(3)) as "struct3B"))
   }
+
+  test("simplify add multiple nested fields to struct") {
+    // this scenario is possible if users add multiple nested columns via the Column.withField API
+    // ideally, users should not be doing this.
+    val nullableStructLevel2 = LocalRelation(
+      'a1.struct(
+        'a2.struct('a3.int)).withNullability(false))
+
+    val query = {
+      val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2",
+        UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2)))))))
+
+      nullableStructLevel2.select(
+        UpdateFields(
+          addB3toA1A2,
+          Seq(WithField("a2", UpdateFields(
+            GetStructField(addB3toA1A2, 0), Seq(WithField("c3", Literal(3))))))).as("a1"))
+    }
+
+    val expected = nullableStructLevel2.select(
+      UpdateFields('a1, Seq(
+        // scalastyle:off line.size.limit
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: Nil)),
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil))
+        // scalastyle:on line.size.limit
+      )).as("a1"))

Review comment:
       This first `WithField` in here is entirely redundant and ideally we would optimize this away as well. 
   However, in the interests of keeping this PR simple, I have opted to forgo writing any such optimizer rule. 
   If necessary, we can address this in a future PR. 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       should I use `semanticEquals` or `fastEquals` here? The difference isn't entirely clear to me and my tests seem to pass in either scenario. 

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *

Review comment:
       One of the issues in master branch with the current `Column.withField` implementation is the size of the parsed logical plan scales non-linearly with the number of directly-add-**nested**-column operations. This results in the driver spending a considerable amount of time analyzing and optimizing the logical plan (literally minutes, if it ever completes). 
   Users can avoid this issue entirely by writing their queries in a performant manner. 
   For example: 
   
   ```
     lazy val nullableStructLevel2: DataFrame = spark.createDataFrame(
       sparkContext.parallelize(Row(Row(Row(0))) :: Nil),
       StructType(Seq(
         StructField("a1", StructType(Seq(
           StructField("a2", StructType(Seq(
             StructField("col0", IntegerType, nullable = false))),
             nullable = true))),
           nullable = true))))
   
     val numColsToAdd = 100
   
     val expectedRows = Row(Row(Row(0 to numColsToAdd: _*))) :: Nil
     val expectedSchema =
       StructType(Seq(
         StructField("a1", StructType(Seq(
           StructField("a2", StructType((0 to numColsToAdd).map(num =>
             StructField(s"col$num", IntegerType, nullable = false))),
             nullable = true))),
           nullable = true)))
   
     test("good way of writing query") {
       // Spark can easily analyze and optimize the parsed logical plan in seconds
       checkAnswer(
         nullableStructLevel2
           .select(col("a1").withField("a2", (1 to numColsToAdd).foldLeft(col("a1.a2")) {
             (column, num) => column.withField(s"col$num", lit(num))
           }).as("a1")),
         expectedRows,
         expectedSchema)
     }
   
     test("bad way of writing the same query that will eventually fail with timeout exception with as little as numColsToAdd = 10") {
       checkAnswer(
         nullableStructLevel2
           .select((1 to numColsToAdd).foldLeft(col("a1")) {
             (column, num) => column.withField(s"a2.col$num", lit(num))
           }.as("a1")),
         expectedRows,
         expectedSchema)
     }
   ```
   
   This issue and its solution is what I've attempted to capture here as part of the method doc. 
   
   There are other options here instead of method-doc-note: 
   - We could potentially write some kind of optimization in `updateFieldsHelper` (I've bashed my head against this for a while but haven't been able to come up with anything satisfactory).
   - Remove the ability to change nested fields directly entirely. While this has the advantage that there will be absolutely no way to run into this "performance" issue, the user-experience definitely suffers for more advanced users who would know how to use these methods properly.  
   
   I've gone with what made most sense to me (method-doc-note) but am open to hearing other people's thoughts on the matter. 
   




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697098161






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698682551


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33719/
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010182


   **[Test build #128995 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128995/testReport)** for PR 29795 at commit [`650d366`](https://github.com/apache/spark/commit/650d366b71982ff496b6f57af66fcf82d77603bf).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492190242



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(

Review comment:
       If it's only used in one test suite, let's move it there. We can move it to `QueryTest` if we see more and more suites using 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701016229






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701106661






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494700714



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("c"))
+   *   // result: {"a":1,"b":2}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b", "c"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("a", "b"))
+   *   // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct
+   *
+   *   val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: null of type struct<a:int>
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b"))
+   *   // result: {"a":{"a":1}}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.c"))
+   *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+   * }}}
+   *
+   * This method supports dropping multiple nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b", "a.c"))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * However, if you are going to drop multiple nested fields, it is more optimal to extract
+   * out the nested struct before dropping multiple fields from it e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * @group expr_ops
+   * @since 3.1.0
+   */
+  // scalastyle:on line.size.limit
+  def dropFields(fieldNames: String*): Column = withExpr {
+    def dropField(structExpr: Expression, fieldName: String): UpdateFields =
+      updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name))
+
+    fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) {
+      (resExpr, fieldName) => dropField(resExpr, fieldName)
+    }
+  }
+
+  private def nameParts(fieldName: String): Seq[String] = {
+    require(fieldName != null, "fieldName cannot be null")
+
+    if (fieldName.isEmpty) {
       fieldName :: Nil

Review comment:
       we've discussed this before [here](https://github.com/apache/spark/pull/27066#discussion_r448416127) :)
   Its needed for `withField` and I think we should support it in `dropFields` as well because `Dataset.drop` supports it: 
   ```
   scala> Seq((1, 2)).toDF("a", "").drop("").printSchema
   root
    |-- a: integer (nullable = false)
   ```
   I've added a test case to demonstrate this works on the `dropFields` side but otherwise left the code unchanged. 
   

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("c"))
+   *   // result: {"a":1,"b":2}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b", "c"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+   *   df.select($"struct_col".dropFields("a", "b"))
+   *   // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct
+   *
+   *   val df = sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: null of type struct<a:int>
+   *
+   *   val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+   *   df.select($"struct_col".dropFields("b"))
+   *   // result: {"a":1}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b"))
+   *   // result: {"a":{"a":1}}
+   *
+   *   val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.c"))
+   *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
+   * }}}
+   *
+   * This method supports dropping multiple nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".dropFields("a.b", "a.c"))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * However, if you are going to drop multiple nested fields, it is more optimal to extract
+   * out the nested struct before dropping multiple fields from it e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c")))
+   *   // result: {"a":{"a":1}}
+   * }}}
+   *
+   * @group expr_ops
+   * @since 3.1.0
+   */
+  // scalastyle:on line.size.limit
+  def dropFields(fieldNames: String*): Column = withExpr {
+    def dropField(structExpr: Expression, fieldName: String): UpdateFields =
+      updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name))
+
+    fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) {
+      (resExpr, fieldName) => dropField(resExpr, fieldName)
+    }
+  }
+
+  private def nameParts(fieldName: String): Seq[String] = {
+    require(fieldName != null, "fieldName cannot be null")
+
+    if (fieldName.isEmpty) {
       fieldName :: Nil
     } else {
       CatalystSqlParser.parseMultipartIdentifier(fieldName)
     }
-    withFieldHelper(expr, nameParts, Nil, col.expr)
   }
 
-  private def withFieldHelper(
-      struct: Expression,
-      namePartsRemaining: Seq[String],
-      namePartsDone: Seq[String],
-      value: Expression) : WithFields = {
-    val name = namePartsRemaining.head
+  private def updateFieldsHelper(
+    structExpr: Expression,

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
##########
@@ -159,6 +160,14 @@ abstract class QueryTest extends PlanTest {
     checkAnswer(df, expectedAnswer.collect())
   }
 
+  protected def checkAnswer(
+    df: => DataFrame,
+    expectedAnswer: Seq[Row],

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {
+    if (depths.length == 1) {
+      StructType(colNums.map { colNum =>
+        StructField(nestedColName(depths.head, colNum), IntegerType, nullable = false)
+      })
+    } else {
+      val depth = depths.head
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(depths.tail, colNums, nullable)
+          structFields :+ StructField(nestedColName(depth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          structFields :+ StructField(nestedColName(depth, colNum), IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(depths: Seq[Int], colNums: Seq[Int]): Row = {
+    if (depths.length == 1) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 => values :+ nestedRow(depths.tail, colNums)
+        case (values, colNum) => values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param depth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  private def nestedDf(
+    depth: Int, numColsAtEachDepth: Int, nullable: Boolean = false): DataFrame = {

Review comment:
       done

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+    depths: Seq[Int], colNums: Seq[Int], nullable: Boolean): StructType = {
+    if (depths.length == 1) {
+      StructType(colNums.map { colNum =>
+        StructField(nestedColName(depths.head, colNum), IntegerType, nullable = false)
+      })
+    } else {
+      val depth = depths.head
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(depths.tail, colNums, nullable)
+          structFields :+ StructField(nestedColName(depth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          structFields :+ StructField(nestedColName(depth, colNum), IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(depths: Seq[Int], colNums: Seq[Int]): Row = {
+    if (depths.length == 1) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 => values :+ nestedRow(depths.tail, colNums)
+        case (values, colNum) => values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param depth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  private def nestedDf(
+    depth: Int, numColsAtEachDepth: Int, nullable: Boolean = false): DataFrame = {
+    require(depth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val depths = 1 to depth
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(depths, colNums)
+    val nestedColumnDataType = nestedStructType(depths, colNums, nullable)
+
+    spark.createDataFrame(
+      sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  test("nestedDf should generate nested DataFrames") {
+    checkAnswer(
+      nestedDf(1, 1),
+      Row(Row(0)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(1, 2),
+      Row(Row(0, 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", IntegerType, nullable = false),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 1),
+      Row(Row(Row(0))) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false))),
+          nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 2),
+      Row(Row(Row(0, 1), 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false),
+          StructField("nested2Col1", IntegerType, nullable = false))),
+          nullable = false),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = false))))
+
+    checkAnswer(
+      nestedDf(2, 2, nullable = true),
+      Row(Row(Row(0, 1), 1)) :: Nil,
+      StructType(Seq(StructField("nested0Col0", StructType(Seq(
+        StructField("nested1Col0", StructType(Seq(
+          StructField("nested2Col0", IntegerType, nullable = false),
+          StructField("nested2Col1", IntegerType, nullable = false))),
+          nullable = true),
+        StructField("nested1Col1", IntegerType, nullable = false))),
+        nullable = true))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  private def addDropNestedColumns(
+    column: Column,

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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494701237



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsPerformanceSuite.scala
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+class UpdateFieldsPerformanceSuite extends QueryTest with SharedSparkSession {

Review comment:
       done, please take a closer look at this as this was relatively new 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494762975



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {

Review comment:
       so the `nullable` only controls the top-level 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698688257


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492559088



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       I mean something like
   ```
   case GetStructField(updateFields: UpdateFields, ordinal, _) if !updateFields.structExpr.isInstanceOf[UpdateFields] =>
     ...
   ```
   As we know that there must be a rule to merge adjacent `UpdateFields` and this rule can wait for it to happen. They are still in the same batch.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r495620066



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {

Review comment:
       no, the nullable parameter affects the top-level and nested StructType 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494764849



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()

Review comment:
       if we only care about analyzer/optimizer performance, we can probably do `nonNullableInputDf.select(columnFunc).queryExecution.optimizedPlan`




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694645847






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010729






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854224


   **[Test build #128923 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128923/testReport)** for PR 29795 at commit [`2f16213`](https://github.com/apache/spark/commit/2f16213ea0a658f481f8f745d759a813fc844dfc).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r496689723



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.addCase("Nullable StructTypes") { _ =>
+        nullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.run()
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    val maxDepth = 20
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 5,
+      numsToAdd = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToDrop = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToAdd = 10 to 14,
+      numsToDrop = 5 to 9)
+  }
+}
+
+class UpdateFieldsBenchmark extends QueryTest with SharedSparkSession {

Review comment:
       Then let's turn it into test, not benchmark. Can we move the tests to `ColumnExpressionSuite`?




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694645847






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701106661






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698674623


   **[Test build #129098 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129098/testReport)** for PR 29795 at commit [`53d83b6`](https://github.com/apache/spark/commit/53d83b635b6e840eb1a66f1c89b453729f589500).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494698769



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {

Review comment:
       thanks for sharing the code, 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694645071


   **[Test build #128840 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128840/testReport)** for PR 29795 at commit [`ac149a5`](https://github.com/apache/spark/commit/ac149a52b88e7eb9ab5aa50d1cf341caf9f59faa).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694556609


   **[Test build #128840 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128840/testReport)** for PR 29795 at commit [`ac149a5`](https://github.com/apache/spark/commit/ac149a52b88e7eb9ab5aa50d1cf341caf9f59faa).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r490633841



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}

Review comment:
       @viirya I'm not quite done with this PR yet but I wanted to share it with you early because some of the changes I'm making in here may be helpful for #29587 (assuming this PR is accepted). Specifically, it would be possible to implement sorting of fields in a struct simply by: 
   ```
   case class OrderStructFieldsByName() extends StructFieldsOperation {
     override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] =
       values.sortBy { case (field, _) => field.name }
   }
   
   UpdateFields(structExpr, OrderStructFieldsByName() :: Nil)
   ``` 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701016229






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-701106126


   **[Test build #129251 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129251/testReport)** for PR 29795 at commit [`7e51f35`](https://github.com/apache/spark/commit/7e51f35580db72fda11153d76caf232b83e617cd).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  trait ModifyNestedColumns `


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699696585


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33771/
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494764069



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.addCase("Nullable StructTypes") { _ =>
+        nullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.run()
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    val maxDepth = 20
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 5,
+      numsToAdd = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToDrop = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToAdd = 10 to 14,
+      numsToDrop = 5 to 9)
+  }
+}
+
+class UpdateFieldsBenchmark extends QueryTest with SharedSparkSession {

Review comment:
       do we still need 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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698746139






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854450






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698745385


   **[Test build #129098 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129098/testReport)** for PR 29795 at commit [`53d83b6`](https://github.com/apache/spark/commit/53d83b635b6e840eb1a66f1c89b453729f589500).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854224


   **[Test build #128923 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128923/testReport)** for PR 29795 at commit [`2f16213`](https://github.com/apache/spark/commit/2f16213ea0a658f481f8f745d759a813fc844dfc).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697098161






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854224






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694556609


   **[Test build #128840 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128840/testReport)** for PR 29795 at commit [`ac149a5`](https://github.com/apache/spark/commit/ac149a52b88e7eb9ab5aa50d1cf341caf9f59faa).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492435726



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       good to know, cheers




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699698343






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695895695


   **[Test build #128923 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128923/testReport)** for PR 29795 at commit [`2f16213`](https://github.com/apache/spark/commit/2f16213ea0a658f481f8f745d759a813fc844dfc).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697097522


   **[Test build #128995 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128995/testReport)** for PR 29795 at commit [`650d366`](https://github.com/apache/spark/commit/650d366b71982ff496b6f57af66fcf82d77603bf).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494700403



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.

Review comment:
       I've made this change but now that I think about it, I don't think its actually classifies as a "noop". We still reconstruct the struct unfortunately e.g.
   ```
   val structType = StructType(Seq(
       StructField("a", IntegerType, nullable = false),
       StructField("b", IntegerType, nullable = true),
       StructField("c", IntegerType, nullable = false)))
   
   val structLevel1: DataFrame = spark.createDataFrame(
       sparkContext.parallelize(Row(Row(1, null, 3)) :: Nil),
       StructType(Seq(StructField("a", structType, nullable = false))))
   
   structLevel1.withColumn("a", 'a.dropFields("d")).explain()
   
   == Physical Plan ==
   *(1) Project [named_struct(a, a#1.a, b, a#1.b, c, a#1.c) AS a#3]
   +- *(1) Scan ExistingRDD[a#1]
   ```
   Should I revert this?




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r491753622



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
##########
@@ -537,18 +662,75 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper {
       query(testStructRelation),
       testStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           Literal(2) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           Literal(3) as "struct3B"))
 
     checkRule(
       query(testNullableStructRelation),
       testNullableStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(3)) as "struct3B"))
   }
+
+  test("simplify add multiple nested fields to struct") {
+    // this scenario is possible if users add multiple nested columns via the Column.withField API
+    // ideally, users should not be doing this.
+    val nullableStructLevel2 = LocalRelation(
+      'a1.struct(
+        'a2.struct('a3.int)).withNullability(false))
+
+    val query = {
+      val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2",
+        UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2)))))))
+
+      nullableStructLevel2.select(
+        UpdateFields(
+          addB3toA1A2,
+          Seq(WithField("a2", UpdateFields(
+            GetStructField(addB3toA1A2, 0), Seq(WithField("c3", Literal(3))))))).as("a1"))
+    }
+
+    val expected = nullableStructLevel2.select(
+      UpdateFields('a1, Seq(
+        // scalastyle:off line.size.limit
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: Nil)),
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil))
+        // scalastyle:on line.size.limit
+      )).as("a1"))
+
+    checkRule(query, expected)
+  }
+
+  test("simplify drop multiple nested fields in struct") {
+    // this scenario is possible if users drop multiple nested columns via the Column.dropFields API
+    // ideally, users should not be doing this.
+    val df = LocalRelation(
+      'a1.struct(
+        'a2.struct('a3.int, 'b3.int, 'c3.int).withNullability(false)
+      ).withNullability(false))
+
+    val query = {
+      val dropA1A2B = UpdateFields('a1, Seq(WithField("a2", UpdateFields(
+        GetStructField('a1, 0), Seq(DropField("b3"))))))
+
+      df.select(
+        UpdateFields(
+          dropA1A2B,
+          Seq(WithField("a2", UpdateFields(
+            GetStructField(dropA1A2B, 0), Seq(DropField("c3")))))).as("a1"))
+    }
+
+    val expected = df.select(
+      UpdateFields('a1, Seq(
+        WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3")))),
+        WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3"), DropField("c3"))))
+      )).as("a1"))

Review comment:
       This first `WithField` in here is entirely redundant as well and ideally we would optimize this away as well.
   However, in the interests of keeping this PR simple, I have opted to forgo writing any such optimizer rule.
   If necessary, we can address this in a future PR.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
##########
@@ -1514,27 +1516,578 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
       StructType(Seq(StructField("a", structType, nullable = true))))
 
     // extract newly added field
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("d", lit(4)).getField("d")),
       Row(4) :: Row(null) :: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
 
     // extract newly replaced field
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("a", lit(4)).getField("a")),
       Row(4) :: Row(null):: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
 
     // add new field, extract another field from original struct
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("d", lit(4)).getField("c")),
       Row(3) :: Row(null):: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
 
     // replace field, extract another field from original struct
-    checkAnswerAndSchema(
+    checkAnswer(
       df.withColumn("a", $"a".withField("a", lit(4)).getField("c")),
       Row(3) :: Row(null):: Nil,
       StructType(Seq(StructField("a", IntegerType, nullable = true))))
   }
+
+
+  test("dropFields should throw an exception if called on a non-StructType column") {
+    intercept[AnalysisException] {
+      testData.withColumn("key", $"key".dropFields("a"))
+    }.getMessage should include("struct argument should be struct type, got: int")
+  }
+
+  test("dropFields should throw an exception if fieldName argument is null") {
+    intercept[IllegalArgumentException] {
+      structLevel1.withColumn("a", $"a".dropFields(null))
+    }.getMessage should include("fieldName cannot be null")
+  }
+
+  test("dropFields should throw an exception if any intermediate structs don't exist") {
+    intercept[AnalysisException] {
+      structLevel2.withColumn("a", 'a.dropFields("x.b"))
+    }.getMessage should include("No such struct field x in a")
+
+    intercept[AnalysisException] {
+      structLevel3.withColumn("a", 'a.dropFields("a.x.b"))
+    }.getMessage should include("No such struct field x in a")
+  }
+
+  test("dropFields should throw an exception if intermediate field is not a struct") {
+    intercept[AnalysisException] {
+      structLevel1.withColumn("a", 'a.dropFields("b.a"))
+    }.getMessage should include("struct argument should be struct type, got: int")
+  }
+
+  test("dropFields should throw an exception if intermediate field reference is ambiguous") {
+    intercept[AnalysisException] {
+      val structLevel2: DataFrame = spark.createDataFrame(
+        sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil),
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", structType, nullable = false),
+            StructField("a", structType, nullable = false))),
+            nullable = false))))
+
+      structLevel2.withColumn("a", 'a.dropFields("a.b"))
+    }.getMessage should include("Ambiguous reference to fields")
+  }
+
+  test("dropFields should drop field in struct") {
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("b")),
+      Row(Row(1, 3)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in null struct") {
+    checkAnswer(
+      nullStructLevel1.withColumn("a", $"a".dropFields("b")),
+      Row(null) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = true))))
+  }
+
+  test("dropFields should drop multiple fields in struct") {
+    Seq(
+      structLevel1.withColumn("a", $"a".dropFields("b", "c")),
+      structLevel1.withColumn("a", 'a.dropFields("b").dropFields("c"))
+    ).foreach { df =>
+      checkAnswer(
+        df,
+        Row(Row(1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should throw an exception if no fields will be left in struct") {
+    intercept[AnalysisException] {
+      structLevel1.withColumn("a", 'a.dropFields("a", "b", "c"))
+    }.getMessage should include("cannot drop all fields in struct")
+  }
+
+  test("dropFields should drop field in nested struct") {
+    checkAnswer(
+      structLevel2.withColumn("a", 'a.dropFields("a.b")),
+      Row(Row(Row(1, 3))) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop multiple fields in nested struct") {
+    checkAnswer(
+      structLevel2.withColumn("a", 'a.dropFields("a.b", "a.c")),
+      Row(Row(Row(1))) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in nested null struct") {
+    checkAnswer(
+      nullStructLevel2.withColumn("a", $"a".dropFields("a.b")),
+      Row(Row(null)) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = true))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop multiple fields in nested null struct") {
+    checkAnswer(
+      nullStructLevel2.withColumn("a", $"a".dropFields("a.b", "a.c")),
+      Row(Row(null)) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = true))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in deeply nested struct") {
+    checkAnswer(
+      structLevel3.withColumn("a", 'a.dropFields("a.a.b")),
+      Row(Row(Row(Row(1, 3)))) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", StructType(Seq(
+              StructField("a", IntegerType, nullable = false),
+              StructField("c", IntegerType, nullable = false))),
+              nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop all fields with given name in struct") {
+    val structLevel1 = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(1, 2, 3)) :: Nil),
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("b")),
+      Row(Row(1)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop field in struct even if casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") {
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")),
+        Row(Row(1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("B", IntegerType, nullable = false))),
+            nullable = false))))
+
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")),
+        Row(Row(1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should not drop field in struct because casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")),
+        Row(Row(1, 1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("B", IntegerType, nullable = false))),
+            nullable = false))))
+
+      checkAnswer(
+        mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")),
+        Row(Row(1, 1)) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("B", IntegerType, nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should drop nested field in struct even if casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") {
+      checkAnswer(
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a")),
+        Row(Row(Row(1), Row(1, 1))) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("A", StructType(Seq(
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false),
+            StructField("B", StructType(Seq(
+              StructField("a", IntegerType, nullable = false),
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false))),
+            nullable = false))))
+
+      checkAnswer(
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a")),
+        Row(Row(Row(1, 1), Row(1))) :: Nil,
+        StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", StructType(Seq(
+              StructField("a", IntegerType, nullable = false),
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false),
+            StructField("b", StructType(Seq(
+              StructField("b", IntegerType, nullable = false))),
+              nullable = false))),
+            nullable = false))))
+    }
+  }
+
+  test("dropFields should throw an exception because casing is different") {
+    withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") {
+      intercept[AnalysisException] {
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a"))
+      }.getMessage should include("No such struct field A in a, B")
+
+      intercept[AnalysisException] {
+        mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a"))
+      }.getMessage should include("No such struct field b in a, B")
+    }
+  }
+
+  test("dropFields should drop only fields that exist") {
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("d")),
+      Row(Row(1, null, 3)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = true),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      structLevel1.withColumn("a", 'a.dropFields("b", "d")),
+      Row(Row(1, 3)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("c", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      structLevel2.withColumn("a", $"a".dropFields("a.b", "a.d")),
+      Row(Row(Row(1, 3))) :: Nil,
+      StructType(
+        Seq(StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields should drop multiple fields at arbitrary levels of nesting in a single call") {
+    val df: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil),
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", structType, nullable = false),
+          StructField("b", IntegerType, nullable = false))),
+          nullable = false))))
+
+    checkAnswer(
+      df.withColumn("a", $"a".dropFields("a.b", "b")),
+      Row(Row(Row(1, 3))) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))), nullable = false))),
+          nullable = false))))
+  }
+
+  test("dropFields user-facing examples") {
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+        .select($"struct_col".dropFields("b")),
+      Row(Row(1)))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+        .select($"struct_col".dropFields("c")),
+      Row(Row(1, 2)))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col")
+        .select($"struct_col".dropFields("b", "c")),
+      Row(Row(1)))
+
+    intercept[AnalysisException] {
+      sql("SELECT named_struct('a', 1, 'b', 2) struct_col")
+        .select($"struct_col".dropFields("a", "b"))
+    }.getMessage should include("cannot drop all fields in struct")
+
+    checkAnswer(
+      sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
+        .select($"struct_col".dropFields("b")),
+      Row(null))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col")
+        .select($"struct_col".dropFields("b")),
+      Row(Row(1)))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+        .select($"struct_col".dropFields("a.b")),
+      Row(Row(Row(1))))
+
+    intercept[AnalysisException] {
+      sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col")
+        .select($"struct_col".dropFields("a.c"))
+    }.getMessage should include("Ambiguous reference to fields")
+
+    checkAnswer(
+      sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col")
+        .select($"struct_col".dropFields("a.b", "a.c")),
+      Row(Row(Row(1))))
+
+    checkAnswer(
+      sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col")
+        .select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c"))),
+      Row(Row(Row(1))))
+  }
+
+  test("should correctly handle different dropField + withField + getField combinations") {
+    val structType = StructType(Seq(
+      StructField("a", IntegerType, nullable = false),
+      StructField("b", IntegerType, nullable = false)))
+
+    val structLevel1: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(1, 2)) :: Nil),
+      StructType(Seq(StructField("a", structType, nullable = false))))
+
+    val nullStructLevel1: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(null) :: Nil),
+      StructType(Seq(StructField("a", structType, nullable = true))))
+
+    val nullableStructLevel1: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(1, 2)) :: Row(null) :: Nil),
+      StructType(Seq(StructField("a", structType, nullable = true))))
+
+    def check(
+      fieldOps: Column => Column,
+      getFieldName: String,
+      expectedValue: Option[Int]): Unit = {
+
+      def query(df: DataFrame): DataFrame =
+        df.select(fieldOps(col("a")).getField(getFieldName).as("res"))
+
+      checkAnswer(
+        query(structLevel1),
+        Row(expectedValue.orNull) :: Nil,
+        StructType(Seq(StructField("res", IntegerType, nullable = expectedValue.isEmpty))))
+
+      checkAnswer(
+        query(nullStructLevel1),
+        Row(null) :: Nil,
+        StructType(Seq(StructField("res", IntegerType, nullable = true))))
+
+      checkAnswer(
+        query(nullableStructLevel1),
+        Row(expectedValue.orNull) :: Row(null) :: Nil,
+        StructType(Seq(StructField("res", IntegerType, nullable = true))))
+    }
+
+    // add attribute, extract an attribute from the original struct
+    check(_.withField("c", lit(3)), "a", Some(1))
+    check(_.withField("c", lit(3)), "b", Some(2))
+
+    // add attribute, extract added attribute
+    check(_.withField("c", lit(3)), "c", Some(3))
+    check(_.withField("c", col("a.a")), "c", Some(1))
+    check(_.withField("c", col("a.b")), "c", Some(2))
+    check(_.withField("c", lit(null).cast(IntegerType)), "c", None)
+
+    // replace attribute, extract an attribute from the original struct
+    check(_.withField("b", lit(3)), "a", Some(1))
+    check(_.withField("a", lit(3)), "b", Some(2))
+
+    // replace attribute, extract replaced attribute
+    check(_.withField("b", lit(3)), "b", Some(3))
+    check(_.withField("b", lit(null).cast(IntegerType)), "b", None)
+    check(_.withField("a", lit(3)), "a", Some(3))
+    check(_.withField("a", lit(null).cast(IntegerType)), "a", None)
+
+    // drop attribute, extract an attribute from the original struct
+    check(_.dropFields("b"), "a", Some(1))
+    check(_.dropFields("a"), "b", Some(2))
+
+    // drop attribute, add attribute, extract an attribute from the original struct
+    check(_.dropFields("b").withField("c", lit(3)), "a", Some(1))
+    check(_.dropFields("a").withField("c", lit(3)), "b", Some(2))
+
+    // drop attribute, add another attribute, extract added attribute
+    check(_.dropFields("a").withField("c", lit(3)), "c", Some(3))
+    check(_.dropFields("b").withField("c", lit(3)), "c", Some(3))
+
+    // add attribute, drop attribute, extract an attribute from the original struct
+    check(_.withField("c", lit(3)).dropFields("a"), "b", Some(2))
+    check(_.withField("c", lit(3)).dropFields("b"), "a", Some(1))
+
+    // add attribute, drop another attribute, extract added attribute
+    check(_.withField("c", lit(3)).dropFields("a"), "c", Some(3))
+    check(_.withField("c", lit(3)).dropFields("b"), "c", Some(3))
+
+    // replace attribute, drop same attribute, extract an attribute from the original struct
+    check(_.withField("b", lit(3)).dropFields("b"), "a", Some(1))
+    check(_.withField("a", lit(3)).dropFields("a"), "b", Some(2))
+
+    // add attribute, drop same attribute, extract an attribute from the original struct
+    check(_.withField("c", lit(3)).dropFields("c"), "a", Some(1))
+    check(_.withField("c", lit(3)).dropFields("c"), "b", Some(2))
+
+    // add attribute, drop another attribute, extract added attribute
+    check(_.withField("b", lit(3)).dropFields("a"), "b", Some(3))
+    check(_.withField("a", lit(3)).dropFields("b"), "a", Some(3))
+    check(_.withField("b", lit(null).cast(IntegerType)).dropFields("a"), "b", None)
+    check(_.withField("a", lit(null).cast(IntegerType)).dropFields("b"), "a", None)
+
+    // drop attribute, add same attribute, extract added attribute
+    check(_.dropFields("b").withField("b", lit(3)), "b", Some(3))
+    check(_.dropFields("a").withField("a", lit(3)), "a", Some(3))
+    check(_.dropFields("b").withField("b", lit(null).cast(IntegerType)), "b", None)
+    check(_.dropFields("a").withField("a", lit(null).cast(IntegerType)), "a", None)
+    check(_.dropFields("c").withField("c", lit(3)), "c", Some(3))
+
+    // add attribute, drop same attribute, add same attribute again, extract added attribute
+    check(_.withField("c", lit(3)).dropFields("c").withField("c", lit(4)), "c", Some(4))
+  }
+
+  test("should move field up one level of nesting") {
+    val nullableStructLevel2: DataFrame = spark.createDataFrame(
+      sparkContext.parallelize(Row(Row(null)) :: Row(Row(Row(1, 2, 3))) :: Nil),
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", structType, nullable = true))),
+          nullable = true))))
+
+    // move a field up one level
+    checkAnswer(
+      nullableStructLevel2.select(
+        col("a").withField("b", col("a.a.b")).dropFields("a.b").as("res")),
+      Row(Row(null, null)) ::  Row(Row(Row(1, 3), 2)) :: Nil,
+      StructType(Seq(
+        StructField("res", StructType(Seq(
+          StructField("a", StructType(Seq(
+            StructField("a", IntegerType, nullable = false),
+            StructField("c", IntegerType, nullable = false))),
+            nullable = true),
+          StructField("b", IntegerType, nullable = true))),
+          nullable = true))))
+
+    // move a field up one level and then extract it
+    checkAnswer(
+      nullableStructLevel2.select(col("a").withField("b", col("a.a.b")).getField("b").as("res")),
+      Row(null) :: Row(2) :: Nil,
+      StructType(Seq(StructField("res", IntegerType, nullable = true))))
+  }
+
+  test("should be able to refer to newly added nested column") {
+    intercept[AnalysisException] {
+      structLevel1.select($"a".withField("d", lit(4)).withField("e", $"a.d" + 1).as("a"))
+    }.getMessage should include("No such struct field d in a, b, c")
+
+    checkAnswer(
+      structLevel1
+        .select($"a".withField("d", lit(4)).as("a"))
+        .select($"a".withField("e", $"a.d" + 1).as("a")),
+      Row(Row(1, null, 3, 4, 5)) :: Nil,
+      StructType(Seq(
+        StructField("a", StructType(Seq(
+          StructField("a", IntegerType, nullable = false),
+          StructField("b", IntegerType, nullable = true),
+          StructField("c", IntegerType, nullable = false),
+          StructField("d", IntegerType, nullable = false),
+          StructField("e", IntegerType, nullable = false))),
+          nullable = false))))
+  }

Review comment:
       I don't expect anyone will be surprised or feel that this is wrong but nevertheless, I did want to highlight this behaviour. Same goes for the two tests below. 

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
##########
@@ -537,18 +662,75 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper {
       query(testStructRelation),
       testStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           Literal(2) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           Literal(3) as "struct3B"))
 
     checkRule(
       query(testNullableStructRelation),
       testNullableStructRelation
         .select(
-          GetStructField('struct1, 0, Some("a")) as "struct2A",
+          GetStructField('struct1, 0) as "struct2A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct2B",
-          GetStructField('struct1, 0, Some("a")) as "struct3A",
+          GetStructField('struct1, 0) as "struct3A",
           If(IsNull('struct1), Literal(null, IntegerType), Literal(3)) as "struct3B"))
   }
+
+  test("simplify add multiple nested fields to struct") {
+    // this scenario is possible if users add multiple nested columns via the Column.withField API
+    // ideally, users should not be doing this.
+    val nullableStructLevel2 = LocalRelation(
+      'a1.struct(
+        'a2.struct('a3.int)).withNullability(false))
+
+    val query = {
+      val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2",
+        UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2)))))))
+
+      nullableStructLevel2.select(
+        UpdateFields(
+          addB3toA1A2,
+          Seq(WithField("a2", UpdateFields(
+            GetStructField(addB3toA1A2, 0), Seq(WithField("c3", Literal(3))))))).as("a1"))
+    }
+
+    val expected = nullableStructLevel2.select(
+      UpdateFields('a1, Seq(
+        // scalastyle:off line.size.limit
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: Nil)),
+        WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil))
+        // scalastyle:on line.size.limit
+      )).as("a1"))

Review comment:
       This first `WithField` in here is entirely redundant and ideally we would optimize this away as well. 
   However, in the interests of keeping this PR simple, I have opted to forgo writing any such optimizer rule. 
   If necessary, we can address this in a future PR. 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       should I use `semanticEquals` or `fastEquals` here? The difference isn't entirely clear to me and my tests seem to pass in either scenario. 

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *

Review comment:
       One of the issues in master branch with the current `Column.withField` implementation is the size of the parsed logical plan scales non-linearly with the number of directly-add-**nested**-column operations. This results in the driver spending a considerable amount of time analyzing and optimizing the logical plan (literally minutes, if it ever completes). 
   Users can avoid this issue entirely by writing their queries in a performant manner. 
   For example: 
   
   ```
     lazy val nullableStructLevel2: DataFrame = spark.createDataFrame(
       sparkContext.parallelize(Row(Row(Row(0))) :: Nil),
       StructType(Seq(
         StructField("a1", StructType(Seq(
           StructField("a2", StructType(Seq(
             StructField("col0", IntegerType, nullable = false))),
             nullable = true))),
           nullable = true))))
   
     val numColsToAdd = 100
   
     val expectedRows = Row(Row(Row(0 to numColsToAdd: _*))) :: Nil
     val expectedSchema =
       StructType(Seq(
         StructField("a1", StructType(Seq(
           StructField("a2", StructType((0 to numColsToAdd).map(num =>
             StructField(s"col$num", IntegerType, nullable = false))),
             nullable = true))),
           nullable = true)))
   
     test("good way of writing query") {
       // Spark can easily analyze and optimize the parsed logical plan in seconds
       checkAnswer(
         nullableStructLevel2
           .select(col("a1").withField("a2", (1 to numColsToAdd).foldLeft(col("a1.a2")) {
             (column, num) => column.withField(s"col$num", lit(num))
           }).as("a1")),
         expectedRows,
         expectedSchema)
     }
   
     test("bad way of writing the same query that will eventually fail with timeout exception with as little as numColsToAdd = 10") {
       checkAnswer(
         nullableStructLevel2
           .select((1 to numColsToAdd).foldLeft(col("a1")) {
             (column, num) => column.withField(s"a2.col$num", lit(num))
           }.as("a1")),
         expectedRows,
         expectedSchema)
     }
   ```
   
   This issue and its solution is what I've attempted to capture here as part of the method doc. 
   
   There are other options here instead of method-doc-note: 
   - We could potentially write some kind of optimization in `updateFieldsHelper` (I've bashed my head against this for a while but haven't been able to come up with anything satisfactory).
   - Remove the ability to change nested fields directly entirely. While this has the advantage that there will be absolutely no way to run into this "performance" issue, the user-experience definitely suffers for more advanced users who would know how to use these methods properly.  
   
   I've gone with what made most sense to me (method-doc-note) but am open to hearing other people's thoughts on the matter. 
   

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       IIUC you mean put `CombineUpdateFields` in a separate batch that runs before the `SimplifyExtractValueOps` batch and remove the `ultimateStruct` method?
   
   If so, we'll miss out on some optimizations because after `SimplifyExtractValueOps` runs we might again end up with code containing `UpdateFields(UpdateFields(_, _))`. The `simplify add multiple nested fields to struct` test in `complexTypesSuite` is a good example of a test that will fail in this scenario. 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr

Review comment:
       good to know, cheers




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698674623


   **[Test build #129098 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129098/testReport)** for PR 29795 at commit [`53d83b6`](https://github.com/apache/spark/commit/53d83b635b6e840eb1a66f1c89b453729f589500).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492187683



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       shall we apply this rule after `UpdateFields` are all merged? then we don't need to do `ultimateStruct`.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494763126



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],

Review comment:
       we can pass in `numCols` and do `0 until numCols` inside this method.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699698340


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33771/
   


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-697010182


   **[Test build #128995 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128995/testReport)** for PR 29795 at commit [`650d366`](https://github.com/apache/spark/commit/650d366b71982ff496b6f57af66fcf82d77603bf).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494745836



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.

Review comment:
       It's semantically noop. We can optimize away the struct reconstructing later.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {

Review comment:
       so the `nullable` only controls the top-level column?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],

Review comment:
       we can pass in `numCols` and do `0 until numCols` inside this method.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.addCase("Nullable StructTypes") { _ =>
+        nullableInputDf.select(columnFunc).noop()
+      }
+
+      benchmark.run()
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    val maxDepth = 20
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 5,
+      numsToAdd = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToDrop = 5 to 9)
+
+    updateFieldsBenchmark(
+      maxDepth = maxDepth,
+      initialNumberOfColumns = 10,
+      numsToAdd = 10 to 14,
+      numsToDrop = 5 to 9)
+  }
+}
+
+class UpdateFieldsBenchmark extends QueryTest with SharedSparkSession {

Review comment:
       do we still need it?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()

Review comment:
       if we only care about analyzer/optimizer performance, we can probably do `nonNullableInputDf.select(columnFunc).queryExecution.optimizedPlan`




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698674623


   **[Test build #129098 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129098/testReport)** for PR 29795 at commit [`53d83b6`](https://github.com/apache/spark/commit/53d83b635b6e840eb1a66f1c89b453729f589500).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r495623143



##########
File path: sql/core/benchmarks/UpdateFieldsBenchmark-results.txt
##########
@@ -0,0 +1,26 @@
+================================================================================================
+Add 2 columns and drop 2 columns at 3 different depths of nesting
+================================================================================================
+
+OpenJDK 64-Bit Server VM 1.8.0_212-b03 on Mac OS X 10.14.6
+Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz
+Add 2 columns and drop 2 columns at 3 different depths of nesting:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+-------------------------------------------------------------------------------------------------------------------------------------------------
+To non-nullable StructTypes using performant method                           10             11           2          0.0      Infinity       1.0X
+To nullable StructTypes using performant method                                9             10           1          0.0      Infinity       1.0X
+To non-nullable StructTypes using non-performant method                     2457           2464          10          0.0      Infinity       0.0X
+To nullable StructTypes using non-performant method                        42641          43804        1644          0.0      Infinity       0.0X
+
+
+================================================================================================
+Add 50 columns and drop 50 columns at 100 different depths of nesting
+================================================================================================
+
+OpenJDK 64-Bit Server VM 1.8.0_212-b03 on Mac OS X 10.14.6
+Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz
+Add 50 columns and drop 50 columns at 100 different depths of nesting:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+-----------------------------------------------------------------------------------------------------------------------------------------------------
+To non-nullable StructTypes using performant method                             4595           4927         470          0.0      Infinity       1.0X
+To nullable StructTypes using performant method                                 5185           5516         468          0.0      Infinity       0.9X
+
+

Review comment:
       Changed the benchmark up a little bit so that we can compare the performant and non-performant methods of updating multiple nested columns. 

##########
File path: sql/core/benchmarks/UpdateFieldsBenchmark-results.txt
##########
@@ -0,0 +1,26 @@
+================================================================================================
+Add 2 columns and drop 2 columns at 3 different depths of nesting
+================================================================================================
+
+OpenJDK 64-Bit Server VM 1.8.0_212-b03 on Mac OS X 10.14.6
+Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz
+Add 2 columns and drop 2 columns at 3 different depths of nesting:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+-------------------------------------------------------------------------------------------------------------------------------------------------
+To non-nullable StructTypes using performant method                           10             11           2          0.0      Infinity       1.0X
+To nullable StructTypes using performant method                                9             10           1          0.0      Infinity       1.0X
+To non-nullable StructTypes using non-performant method                     2457           2464          10          0.0      Infinity       0.0X
+To nullable StructTypes using non-performant method                        42641          43804        1644          0.0      Infinity       0.0X

Review comment:
       This last result is pretty bad (43 seconds). 
   It's partially because of the non-performant method and partially because the optimizer rules aren't perfect  in complex nullable StructType scenarios (I've documented these scenarios in this [commit](https://github.com/apache/spark/pull/29795/commits/4fe48b4287c81e73276165453477811211e341d9)). 
   It should be possible to improve the optimizer rules further. I have a couple of simple ideas I'm toying around with but it will take me a while to reason/test if they are safe from a correctness point of view. 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699692664


   **[Test build #129156 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129156/testReport)** for PR 29795 at commit [`cca6f37`](https://github.com/apache/spark/commit/cca6f37d03df6c41561dc2b4cc127ebf1305a8ed).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r495620119



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala
##########
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
+
+/**
+ * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains.
+ *
+ * {{{
+ *   To run this benchmark:
+ *   1. without sbt:
+ *      bin/spark-submit --class <this class> <spark sql test jar>
+ *   2. with sbt:
+ *      build/sbt "sql/test:runMain <this class>"
+ *   3. generate result:
+ *      SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *   Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt".
+ * }}}
+ */
+object UpdateFieldsBenchmark extends SqlBasedBenchmark {
+
+  private def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum"
+
+  private def nestedStructType(
+      colNums: Seq[Int],
+      nullable: Boolean,
+      maxDepth: Int,
+      currDepth: Int = 1): StructType = {
+
+    if (currDepth == maxDepth) {
+      val fields = colNums.map { colNum =>
+        val name = nestedColName(currDepth, colNum)
+        StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    } else {
+      val fields = colNums.foldLeft(Seq.empty[StructField]) {
+        case (structFields, colNum) if colNum == 0 =>
+          val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1)
+          structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable)
+        case (structFields, colNum) =>
+          val name = nestedColName(currDepth, colNum)
+          structFields :+ StructField(name, IntegerType, nullable = false)
+      }
+      StructType(fields)
+    }
+  }
+
+  private def nestedRow(colNums: Seq[Int], maxDepth: Int, currDepth: Int = 1): Row = {
+    if (currDepth == maxDepth) {
+      Row.fromSeq(colNums)
+    } else {
+      val values = colNums.foldLeft(Seq.empty[Any]) {
+        case (values, colNum) if colNum == 0 =>
+          values :+ nestedRow(colNums, maxDepth, currDepth + 1)
+        case (values, colNum) =>
+          values :+ colNum
+      }
+      Row.fromSeq(values)
+    }
+  }
+
+  /**
+   * Utility function for generating a DataFrame with nested columns.
+   *
+   * @param maxDepth: The depth to which to create nested columns.
+   * @param numColsAtEachDepth: The number of columns to create at each depth. The value of each
+   *                          column will be the same as its index (IntegerType) at that depth
+   *                          unless the index = 0, in which case it may be a StructType which
+   *                          represents the next depth.
+   * @param nullable: This value is used to set the nullability of StructType columns.
+   */
+  def nestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = {
+    require(maxDepth > 0)
+    require(numColsAtEachDepth > 0)
+
+    val colNums = 0 until numColsAtEachDepth
+    val nestedColumn = nestedRow(colNums, maxDepth)
+    val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth)
+
+    spark.createDataFrame(
+      spark.sparkContext.parallelize(Row(nestedColumn) :: Nil),
+      StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable))))
+  }
+
+  // simulates how a user would add/drop nested fields in a performant manner
+  def modifyNestedColumns(
+      column: Column,
+      numsToAdd: Seq[Int],
+      numsToDrop: Seq[Int],
+      maxDepth: Int,
+      currDepth: Int = 1): Column = {
+
+    // drop columns at the current depth
+    val dropped = if (numsToDrop.nonEmpty) {
+      column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*)
+    } else column
+
+    // add columns at the current depth
+    val added = numsToAdd.foldLeft(dropped) {
+      (res, num) => res.withField(nestedColName(currDepth, num), lit(num))
+    }
+
+    if (currDepth == maxDepth) {
+      added
+    } else {
+      // add/drop columns at the next depth
+      val newValue = modifyNestedColumns(
+        column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")),
+        numsToAdd = numsToAdd,
+        numsToDrop = numsToDrop,
+        currDepth = currDepth + 1,
+        maxDepth = maxDepth)
+      added.withField(nestedColName(currDepth, 0), newValue)
+    }
+  }
+
+  def updateFieldsBenchmark(
+      maxDepth: Int,
+      initialNumberOfColumns: Int,
+      numsToAdd: Seq[Int] = Seq.empty,
+      numsToDrop: Seq[Int] = Seq.empty): Unit = {
+
+    val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " +
+      s"at $maxDepth different depths of nesting"
+
+    runBenchmark(name) {
+      val benchmark = new Benchmark(
+        name = name,
+        // Because the point of this benchmark is only to ensure Spark is able to analyze and
+        // optimize long UpdateFields chains quickly, this benchmark operates only over 1 row of
+        // data.
+        valuesPerIteration = 1,
+        output = output)
+
+      val columnFunc = modifyNestedColumns(
+        col(nestedColName(0, 0)),
+        numsToAdd,
+        numsToDrop,
+        maxDepth
+      ).as(nestedColName(0, 0))
+
+      val nonNullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = false)
+      val nullableInputDf = nestedDf(maxDepth, initialNumberOfColumns, nullable = true)
+
+      benchmark.addCase("Non-Nullable StructTypes") { _ =>
+        nonNullableInputDf.select(columnFunc).noop()

Review comment:
       nice, updated the code. 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494745836



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/Column.scala
##########
@@ -901,39 +901,125 @@ class Column(val expr: Expression) extends Logging {
    *   // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields
    * }}}
    *
+   * This method supports adding/replacing nested fields directly e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4)))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
+   * However, if you are going to add/replace multiple nested fields, it is more optimal to extract
+   * out the nested struct before adding/replacing multiple fields e.g.
+   *
+   * {{{
+   *   val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col")
+   *   df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4))))
+   *   // result: {"a":{"a":1,"b":2,"c":3,"d":4}}
+   * }}}
+   *
    * @group expr_ops
    * @since 3.1.0
    */
   // scalastyle:on line.size.limit
   def withField(fieldName: String, col: Column): Column = withExpr {
     require(fieldName != null, "fieldName cannot be null")
     require(col != null, "col cannot be null")
+    updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr))
+  }
 
-    val nameParts = if (fieldName.isEmpty) {
+  // scalastyle:off line.size.limit
+  /**
+   * An expression that drops fields in `StructType` by name.

Review comment:
       It's semantically noop. We can optimize away the struct reconstructing later.




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699692664


   **[Test build #129156 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129156/testReport)** for PR 29795 at commit [`cca6f37`](https://github.com/apache/spark/commit/cca6f37d03df6c41561dc2b4cc127ebf1305a8ed).


----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r492435473



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
##########
@@ -39,19 +40,14 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] {
       // Remove redundant field extraction.
       case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) =>
         createNamedStruct.valExprs(ordinal)
-      case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) =>
-        val name = w.dataType(ordinal).name
-        val matches = names.zip(valExprs).filter(_._1 == name)
-        if (matches.nonEmpty) {
-          // return last matching element as that is the final value for the field being extracted.
-          // For example, if a user submits a query like this:
-          // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")`
-          // we want to return `lit(2)` (and not `lit(1)`).
-          val expr = matches.last._2
-          If(IsNull(struct), Literal(null, expr.dataType), expr)
-        } else {
-          GetStructField(struct, ordinal, maybeName)
-        }
+    case GetStructField(updateFields: UpdateFields, ordinal, _) =>
+      val structExpr = updateFields.structExpr
+      updateFields.newExprs(ordinal) match {
+        // if the struct itself is null, then any value extracted from it (expr) will be null
+        // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr)
+        case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr
+        case expr => If(IsNull(ultimateStruct(structExpr)), Literal(null, expr.dataType), expr)

Review comment:
       IIUC you mean put `CombineUpdateFields` in a separate batch that runs before the `SimplifyExtractValueOps` batch and remove the `ultimateStruct` method?
   
   If so, we'll miss out on some optimizations because after `SimplifyExtractValueOps` runs we might again end up with code containing `UpdateFields(UpdateFields(_, _))`. The `simplify add multiple nested fields to struct` test in `complexTypesSuite` is a good example of a test that will fail in this scenario. 




----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699733089






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494698809



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {
+      values.map {
+        case (field, _) if resolver(field.name, name) => newFieldExpr
+        case x => x
+      }
+    } else {
+      values :+ newFieldExpr
+    }
+  }
+
+  override def children: Seq[Expression] = valExpr :: Nil
+
+  override def dataType: DataType = throw new UnresolvedException(this, "dataType")
+
+  override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
+
+  override def prettyName: String = "WithField"
+}
+
+/**
+ * Drop a field by name.
+ */
+case class DropField(name: String) extends StructFieldsOperation {
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] =
+    values.filterNot { case (field, _) => resolver(field.name, name) }
+}
+
+/**
+ * Updates fields in a struct.
+ */
+case class UpdateFields(structExpr: Expression, fieldOps: Seq[StructFieldsOperation])
+  extends Unevaluable {
 
   override def checkInputDataTypes(): TypeCheckResult = {
-    if (!structExpr.dataType.isInstanceOf[StructType]) {
-      TypeCheckResult.TypeCheckFailure(
-        "struct argument should be struct type, got: " + structExpr.dataType.catalogString)
+    val dataType = structExpr.dataType
+    if (!dataType.isInstanceOf[StructType]) {
+      TypeCheckResult.TypeCheckFailure("struct argument should be struct type, got: " +
+        dataType.catalogString)
+    } else if (newExprs.isEmpty) {
+      TypeCheckResult.TypeCheckFailure("cannot drop all fields in struct")
     } else {
       TypeCheckResult.TypeCheckSuccess
     }
   }
 
-  override def children: Seq[Expression] = structExpr +: valExprs
+  override def children: Seq[Expression] = structExpr +: fieldOps.collect {
+    case e: Expression => e
+  }
 
-  override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType]
+  override def dataType: StructType = StructType(newFields)
 
   override def nullable: Boolean = structExpr.nullable
 
-  override def prettyName: String = "with_fields"
+  override def prettyName: String = "update_fields"
 
-  lazy val evalExpr: Expression = {
-    val existingExprs = structExpr.dataType.asInstanceOf[StructType].fieldNames.zipWithIndex.map {
-      case (name, i) => (name, GetStructField(KnownNotNull(structExpr), i).asInstanceOf[Expression])
+  private lazy val existingFieldExprs: Seq[(StructField, Expression)] =
+    structExpr.dataType.asInstanceOf[StructType].fields.zipWithIndex.map {
+      case (field, i) => (field, GetStructField(structExpr, i))
     }
 
-    val addOrReplaceExprs = names.zip(valExprs)
-
-    val resolver = SQLConf.get.resolver
-    val newExprs = addOrReplaceExprs.foldLeft(existingExprs) {
-      case (resultExprs, newExpr @ (newExprName, _)) =>
-        if (resultExprs.exists(x => resolver(x._1, newExprName))) {
-          resultExprs.map {
-            case (name, _) if resolver(name, newExprName) => newExpr
-            case x => x
-          }
-        } else {
-          resultExprs :+ newExpr
-        }
-    }.flatMap { case (name, expr) => Seq(Literal(name), expr) }
+  private lazy val newFieldExprs: Seq[(StructField, Expression)] =
+    fieldOps.foldLeft(existingFieldExprs)((exprs, op) => op(exprs))
 
-    val expr = CreateNamedStruct(newExprs)
-    if (structExpr.nullable) {
-      If(IsNull(structExpr), Literal(null, expr.dataType), expr)
-    } else {
-      expr
-    }
+  private lazy val newFields: Seq[StructField] = newFieldExprs.map(_._1)
+
+  lazy val newExprs: Seq[Expression] = newFieldExprs.map(_._2)
+
+  private lazy val createNamedStructExpr = CreateNamedStruct(newFieldExprs.flatMap {
+    case (field, expr) => Seq(Literal(field.name), expr)
+  })
+
+  lazy val evalExpr: Expression = if (structExpr.nullable) {

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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-698674623






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] fqaiser94 commented on a change in pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
fqaiser94 commented on a change in pull request #29795:
URL: https://github.com/apache/spark/pull/29795#discussion_r494698625



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
##########
@@ -541,57 +541,105 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
 }
 
 /**
- * Adds/replaces field in struct by name.
+ * Represents an operation to be applied to the fields of a struct.
  */
-case class WithFields(
-    structExpr: Expression,
-    names: Seq[String],
-    valExprs: Seq[Expression]) extends Unevaluable {
+trait StructFieldsOperation {
 
-  assert(names.length == valExprs.length)
+  val resolver: Resolver = SQLConf.get.resolver
+
+  /**
+   * Returns an updated list of StructFields and Expressions that will ultimately be used
+   * as the fields argument for [[StructType]] and as the children argument for
+   * [[CreateNamedStruct]] respectively inside of [[UpdateFields]].
+   */
+  def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)]
+}
+
+/**
+ * Add or replace a field by name.
+ *
+ * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its
+ * children, and thereby enable the analyzer to resolve and transform valExpr as necessary.
+ */
+case class WithField(name: String, valExpr: Expression)
+  extends Unevaluable with StructFieldsOperation {
+
+  override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = {
+    val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr)
+    if (values.exists { case (field, _) => resolver(field.name, name) }) {
+      values.map {
+        case (field, _) if resolver(field.name, name) => newFieldExpr
+        case x => x
+      }
+    } else {
+      values :+ newFieldExpr
+    }
+  }
+
+  override def children: Seq[Expression] = valExpr :: Nil
+
+  override def dataType: DataType = throw new UnresolvedException(this, "dataType")

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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-699733089






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854450






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-694554861






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29795: [SPARK-32511][SQL][WIP] Add dropFields method to Column class

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29795:
URL: https://github.com/apache/spark/pull/29795#issuecomment-695854450






----------------------------------------------------------------
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: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org